You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2019/06/19 19:55:38 UTC

[nifi] branch master updated: NIFI-5373 Created new module nifi-metrics In nifi-metrics, added build-helper-maven-plugin to enable groovy tests to run, since there are no java tests or src/test/java path in the module Replaced yammer/dropwizard's VirtualMachineMetrics usage in multiple modules with JvmMetrics interface and JmxJvmMetrics implementation in nifi-metrics Updated nifi-datadog-reporting-task to use nifi-metrics:1.10.0-SNAPSHOT Updated nifi-scripting-bundle to use nifi-metrics:1.10.0-SNAPSHOT Updated Scripted [...]

This is an automated email from the ASF dual-hosted git repository.

mattyb149 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 82a6c43  NIFI-5373 Created new module nifi-metrics In nifi-metrics, added build-helper-maven-plugin to enable groovy tests to run, since there are no java tests or src/test/java path in the module Replaced yammer/dropwizard's VirtualMachineMetrics usage in multiple modules with JvmMetrics interface and JmxJvmMetrics implementation in nifi-metrics Updated nifi-datadog-reporting-task to use nifi-metrics:1.10.0-SNAPSHOT Updated nifi-scripting-bundle to use nifi-metrics:1.10.0-SNAPSH [...]
82a6c43 is described below

commit 82a6c430a4cea116d011991173f867c8bc3fc35e
Author: Jeff Storck <jt...@gmail.com>
AuthorDate: Fri May 31 14:33:24 2019 -0400

    NIFI-5373 Created new module nifi-metrics
    In nifi-metrics, added build-helper-maven-plugin to enable groovy tests to run, since there are no java tests or src/test/java path in the module
    Replaced yammer/dropwizard's VirtualMachineMetrics usage in multiple modules with JvmMetrics interface and JmxJvmMetrics implementation in nifi-metrics
    Updated nifi-datadog-reporting-task to use nifi-metrics:1.10.0-SNAPSHOT
    Updated nifi-scripting-bundle to use nifi-metrics:1.10.0-SNAPSHOT
    Updated ScriptedReportingTask to use JmxJvmMetrics instead of VirtualMachineMetrics
    Updated nifi-reporting-utils pom to use nifi-metrics:1.10.0-SNAPSHOT
    Updated nifi-ambari-reporting-task pom to use nifi-metrics:1.10.0-SNAPSHOT
    Updated SiteToSiteMetricsReportingTask to use nifi-metrics:1.10.0-SNAPSHOT
    Updated ScriptedReportingTaskGroovyTest to use regular strings instead of GStrings due to class cast exception
    Updated nifi-prometheus-reporting-task to use nifi-metrics:1.10.0-SNAPSHOT
    Updated PrometheusMetricsUtil to use LF endings instead of CRLF
    Updated nifi-metrics-reporting-bundle and its submodules to use nifi-metrics:1.10.0-SNAPSHOT
    nifi-metrics-reporting-task metrics-graphite version upgraded from 3.1.2 to 4.1.0
    Removed unused property yammer.metrics.version from nifi-jolt-record-bundle pom
    Updated root NOTICE to include derived work in nifi-metrics
    Removed duplicate nifi-assembly NOTICE clauses for yammer/dropwizard metrics
    Removed unnecessary inclusion of Yammer Metrics in Kafka 0-9, 0-10, 0-11, 1-0, and 2-0 NARs' NOTICE files since the Kafka dependency that transitively includes it is test-scoped.
    Removed unnecessary mock interaction in GraphiteMetricReporterServiceTest, Mockito 2.28.2 operates in strict mode and flags unused unused interactions.
    
    NIFI-6354: Added LICENSE and NOTICE for nifi-kafka-2-0-nar
    Updated NOTICE in kafka nar modules to reflect the actual module names at the top of the NOTICE
    Added yammer metrics NOTICE clauses to nifi-kafka-0-11-nar
    Added yammer metrics NOTICE clauses to nifi-kafka-1-0-nar
    
    NIFI-6355: Extended NOTICE clauses for yammer/dropwizard metrics in multiple modules
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #3520
---
 NOTICE                                             |  16 +
 nifi-assembly/NOTICE                               |  49 +-
 nifi-commons/nifi-metrics/pom.xml                  |  81 +++
 .../org/apache/nifi/metrics/jvm/JmxJvmMetrics.java | 227 ++++++++
 .../org/apache/nifi/metrics/jvm/JvmMetrics.java    | 280 ++++++++++
 .../nifi/metrics/jvm/JmxJvmMetricsSpec.groovy      | 101 ++++
 nifi-commons/pom.xml                               |   1 +
 .../src/main/resources/META-INF/NOTICE             |  15 +-
 .../nifi-ambari-reporting-task/pom.xml             |   5 +
 .../nifi/reporting/ambari/AmbariReportingTask.java |   8 +-
 .../ambari/metrics/TestMetricsService.java         |   5 +-
 .../src/main/resources/META-INF/NOTICE             |  15 +-
 .../src/main/resources/META-INF/NOTICE             |   9 +
 .../src/main/resources/META-INF/NOTICE             | 461 ++++++++--------
 .../src/main/resources/META-INF/NOTICE             | 591 +++++++++++----------
 .../src/main/resources/META-INF/NOTICE             |  15 +-
 .../nifi-datadog-reporting-task/pom.xml            |  13 +-
 .../reporting/datadog/DataDogReportingTask.java    |   6 +-
 .../reporting/datadog/metrics/MetricsService.java  |   9 +-
 .../datadog/TestDataDogReportingTask.java          |   8 +-
 .../nifi/reporting/datadog/TestMetricsService.java |   4 +-
 .../nifi-reporting-utils/pom.xml                   |  11 +-
 .../reporting/util/metrics/MetricsService.java     |  18 +-
 nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml   |   1 -
 .../src/main/resources/META-INF/NOTICE             |  16 +-
 .../src/main/resources/META-INF/NOTICE             |   2 +-
 .../src/main/resources/META-INF/NOTICE             |   2 +-
 .../src/main/resources/META-INF/NOTICE             |  14 -
 .../src/main/resources/META-INF/NOTICE             |   2 +-
 .../src/main/resources/META-INF/LICENSE            | 233 ++++++++
 .../src/main/resources/META-INF/NOTICE             |   2 +-
 .../src/main/resources/META-INF/NOTICE             |   9 +-
 .../nifi-metrics-reporter-service-api/pom.xml      |   7 +
 .../src/main/resources/META-INF/NOTICE             |   9 +-
 .../nifi-metrics-reporting-task/pom.xml            |   7 +-
 .../service/GraphiteMetricReporterServiceTest.java |   2 -
 .../nifi-metrics-reporting-bundle/pom.xml          |  10 +-
 .../src/main/resources/META-INF/NOTICE             |  15 +-
 .../nifi-prometheus-reporting-task/pom.xml         |   9 +-
 .../reporting/prometheus/PrometheusServer.java     |   5 +-
 .../prometheus/api/PrometheusMetricsUtil.java      |   8 +-
 .../src/main/resources/META-INF/NOTICE             |  29 +-
 .../nifi-scripting-processors/pom.xml              |   9 +-
 .../reporting/script/ScriptedReportingTask.java    |   6 +-
 nifi-nar-bundles/nifi-scripting-bundle/pom.xml     |   6 +
 .../src/main/resources/META-INF/NOTICE             |  19 +
 .../reporting/SiteToSiteMetricsReportingTask.java  |   6 +-
 .../src/main/resources/META-INF/NOTICE             |  11 +
 48 files changed, 1692 insertions(+), 695 deletions(-)

diff --git a/NOTICE b/NOTICE
index 508beb8..94a3b97 100644
--- a/NOTICE
+++ b/NOTICE
@@ -91,3 +91,19 @@ This includes derived works from apigateway-generic-java-sdk (ASLv2 licenced) pr
       RequestMatcher.java
       GetAWSGatewayApiTest.java
 
+This includes derived works from Dropwizard Metrics available under Apache Software License V2 (https://github.com/dropwizard/metrics)
+  Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
+  This product includes software developed by Coda Hale and Yammer, Inc.
+
+  This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+  LongAdder), which was released with the following comments:
+
+      Written by Doug Lea with assistance from members of JCP JSR-166
+      Expert Group and released to the public domain, as explained at
+      http://creativecommons.org/publicdomain/zero/1.0/
+
+  The derived work in the nifi-metrics module is adapted from
+    https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+  and can be found in
+    nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+    nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index fc94815..9850ad8 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -47,6 +47,23 @@ This includes derived works from Elastic Logstash (https://github.com/elastic/lo
   The derived work consists in modifications from patterns/grok-patterns
   and can be found in the file nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestExtractGrok/patterns
 
+This includes derived works from Dropwizard Metrics available under Apache Software License V2 (https://github.com/dropwizard/metrics)
+  Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
+  This product includes software developed by Coda Hale and Yammer, Inc.
+
+  This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+  LongAdder), which was released with the following comments:
+
+      Written by Doug Lea with assistance from members of JCP JSR-166
+      Expert Group and released to the public domain, as explained at
+      http://creativecommons.org/publicdomain/zero/1.0/
+
+  The derived work in the nifi-metrics module is adapted from
+    https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+  and can be found in
+    nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+    nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
+
 ===========================================
 Apache Software License v2
 ===========================================
@@ -111,10 +128,6 @@ The following binary components are provided under the Apache Software License v
 
          Tatu Saloranta (http://wiki.fasterxml.com/TatuSaloranta)
 
-  (ASLv2) Dropwizard Metrics
-    The following NOTICE information applies:
-	Copyright (c) 2010-2013 Coda Hale, Yammer.com
-
   (ASLv2) Apache POI
     The following NOTICE information applies:
 
@@ -640,20 +653,6 @@ The following binary components are provided under the Apache Software License v
         FedICT (federal ICT department of Belgium), e-Contract.be BVBA (https://www.e-contract.be),
         Bart Hanssens from FedICT
 
-  (ASLv2) Yammer Metrics
-    The following NOTICE information applies:
-      Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-      This product includes software developed by Coda Hale and Yammer, Inc.
-
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
-
-          Written by Doug Lea with assistance from members of JCP JSR-166
-          Expert Group and released to the public domain, as explained at
-          http://creativecommons.org/publicdomain/zero/1.0/
-
   (ASLv2) Apache CXF
     The following NOTICE information applies:
       Apache CXF
@@ -1212,20 +1211,6 @@ The following binary components are provided under the Apache Software License v
         Progress Software Corporation and/or its  subsidiaries or affiliates.
       * This product includes software developed by IBM Corporation and others.
 
-  (ASLv2) Metrics
-    The following NOTICE information applies:
-      Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-      This product includes software developed by Coda Hale and Yammer, Inc.
-
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
-
-          Written by Doug Lea with assistance from members of JCP JSR-166
-          Expert Group and released to the public domain, as explained at
-          http://creativecommons.org/publicdomain/zero/1.0/
-
   (ASLv2) ParCEFone
     The following NOTICE information applies:
       ParCEFone
diff --git a/nifi-commons/nifi-metrics/pom.xml b/nifi-commons/nifi-metrics/pom.xml
new file mode 100644
index 0000000..e10f013
--- /dev/null
+++ b/nifi-commons/nifi-metrics/pom.xml
@@ -0,0 +1,81 @@
+<?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>
+        <artifactId>nifi-commons</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.10.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-metrics</artifactId>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.5</version>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/groovy</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-test-source</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/groovy</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-jvm</artifactId>
+            <version>4.1.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.spockframework</groupId>
+            <artifactId>spock-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java b/nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
new file mode 100644
index 0000000..55969fc
--- /dev/null
+++ b/nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
@@ -0,0 +1,227 @@
+/*
+ * 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.metrics.jvm;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.JvmAttributeGaugeSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import org.apache.nifi.processor.DataUnit;
+
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+public class JmxJvmMetrics implements JvmMetrics {
+
+    static final String REGISTRY_METRICSET_JVM_ATTRIBUTES = "jvm-attributes";
+    static final String REGISTRY_METRICSET_MEMORY = "memory";
+    public static final String MEMORY_POOLS = REGISTRY_METRICSET_MEMORY + ".pools";
+    static final String REGISTRY_METRICSET_THREADS = "threads";
+    static final String REGISTRY_METRICSET_GARBAGE_COLLECTORS = "garbage-collectors";
+    static final String JVM_ATTRIBUTES_NAME = REGISTRY_METRICSET_JVM_ATTRIBUTES + ".name";
+    static final String JVM_ATTRIBUTES_UPTIME = REGISTRY_METRICSET_JVM_ATTRIBUTES + ".uptime";
+    static final String JVM_ATTRIBUTES_VENDOR = REGISTRY_METRICSET_JVM_ATTRIBUTES + ".vendor";
+    static final String MEMORY_TOTAL_INIT = REGISTRY_METRICSET_MEMORY + ".total.init";
+    static final String MEMORY_TOTAL_USED = REGISTRY_METRICSET_MEMORY + ".total.used";
+    static final String MEMORY_TOTAL_MAX = REGISTRY_METRICSET_MEMORY + ".total.max";
+    static final String MEMORY_TOTAL_COMMITTED = REGISTRY_METRICSET_MEMORY + ".total.committed";
+    static final String MEMORY_HEAP_INIT = REGISTRY_METRICSET_MEMORY + ".heap.init";
+    static final String MEMORY_HEAP_USED = REGISTRY_METRICSET_MEMORY + ".heap.used";
+    static final String MEMORY_HEAP_MAX = REGISTRY_METRICSET_MEMORY + ".heap.max";
+    static final String MEMORY_HEAP_COMMITTED = REGISTRY_METRICSET_MEMORY + ".heap.committed";
+    static final String MEMORY_HEAP_USAGE = REGISTRY_METRICSET_MEMORY + ".heap.usage";
+    static final String MEMORY_NON_HEAP_USAGE = REGISTRY_METRICSET_MEMORY + ".non-heap.usage";
+    static final String THREADS_COUNT = REGISTRY_METRICSET_THREADS + ".count";
+    static final String THREADS_DAEMON_COUNT = REGISTRY_METRICSET_THREADS + ".daemon.count";
+    static final String THREADS_DEADLOCKS = REGISTRY_METRICSET_THREADS + ".deadlocks";
+    static final String OS_FILEDESCRIPTOR_USAGE = "os.filedescriptor.usage";
+
+    private static AtomicReference<MetricRegistry> metricRegistry = new AtomicReference<>(null);
+
+    private JmxJvmMetrics() {
+    }
+
+    public static JmxJvmMetrics getInstance() {
+        if (metricRegistry.get() == null) {
+            metricRegistry.set(new MetricRegistry());
+            metricRegistry.get().register(REGISTRY_METRICSET_JVM_ATTRIBUTES, new JvmAttributeGaugeSet());
+            metricRegistry.get().register(REGISTRY_METRICSET_MEMORY, new MemoryUsageGaugeSet());
+            metricRegistry.get().register(REGISTRY_METRICSET_THREADS, new ThreadStatesGaugeSet());
+            metricRegistry.get().register(REGISTRY_METRICSET_GARBAGE_COLLECTORS, new GarbageCollectorMetricSet());
+            metricRegistry.get().register(OS_FILEDESCRIPTOR_USAGE, new FileDescriptorRatioGauge());
+
+        }
+        return new JmxJvmMetrics();
+    }
+
+    private Object getMetric(String metricName) {
+        final SortedMap<String, Gauge> gauges = metricRegistry.get().getGauges((name, metric) -> name.equals(metricName));
+        if (gauges.isEmpty()) {
+            throw new IllegalArgumentException(String.format("Unable to retrieve metric \"%s\"", metricName));
+        }
+        return gauges.get(metricName).getValue();
+    }
+
+    public Set<String> getMetricNames(String metricNamePrefix) {
+        if (metricNamePrefix == null || metricNamePrefix.length() == 0) {
+            throw new IllegalArgumentException("A metric name prefix must be supplied");
+        }
+        final String normalizedMetricNamePrefix = metricNamePrefix.endsWith(".") ? metricNamePrefix : metricNamePrefix + '.';
+        return metricRegistry.get().getNames().stream()
+                .filter(name -> name.startsWith(normalizedMetricNamePrefix))
+                .map(name -> name.substring(normalizedMetricNamePrefix.length(), name.indexOf(".", normalizedMetricNamePrefix.length())))
+                .collect(Collectors.toSet());
+    }
+
+    @Override
+    public double totalInit(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_TOTAL_INIT), DataUnit.B);
+    }
+
+    @Override
+    public double totalUsed(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_TOTAL_USED), DataUnit.B);
+    }
+
+    @Override
+    public double totalMax(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_TOTAL_MAX), DataUnit.B);
+    }
+
+    @Override
+    public double totalCommitted(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_TOTAL_COMMITTED), DataUnit.B);
+    }
+
+    @Override
+    public double heapInit(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_HEAP_INIT), DataUnit.B);
+    }
+
+    @Override
+    public double heapUsed(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_HEAP_USED), DataUnit.B);
+    }
+
+    @Override
+    public double heapMax(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_HEAP_MAX), DataUnit.B);
+    }
+
+    @Override
+    public double heapCommitted(DataUnit dataUnit) {
+        return (dataUnit == null ? DataUnit.B : dataUnit).convert((Long) getMetric(MEMORY_HEAP_COMMITTED), DataUnit.B);
+    }
+
+    @Override
+    public double heapUsage() {
+        return (Double) getMetric(MEMORY_HEAP_USAGE);
+    }
+
+    @Override
+    public double nonHeapUsage() {
+        return (Double) getMetric(MEMORY_NON_HEAP_USAGE);
+    }
+
+    @Override
+    public Map<String, Double> memoryPoolUsage() {
+        Set<String> poolNames = getMetricNames(MEMORY_POOLS);
+        Map<String, Double> memoryPoolUsage = new HashMap<>();
+        for (String poolName : poolNames) {
+            memoryPoolUsage.put(poolName, (Double) getMetric(MEMORY_POOLS + "." + poolName + ".usage"));
+        }
+        return Collections.unmodifiableMap(memoryPoolUsage);
+    }
+
+    @Override
+    public double fileDescriptorUsage() {
+        return (Double) getMetric(OS_FILEDESCRIPTOR_USAGE);
+    }
+
+    @Override
+    public String version() {
+        return (String) getMetric(JVM_ATTRIBUTES_VENDOR);
+    }
+
+    @Override
+    public String name() {
+        return (String) getMetric(JVM_ATTRIBUTES_NAME);
+    }
+
+    @Override
+    public long uptime() {
+        return TimeUnit.MILLISECONDS.toSeconds((Long) getMetric(JVM_ATTRIBUTES_UPTIME));
+    }
+
+    @Override
+    public int threadCount() {
+        return (Integer) getMetric(THREADS_COUNT);
+    }
+
+    @Override
+    public int daemonThreadCount() {
+        return (Integer) getMetric(THREADS_DAEMON_COUNT);
+    }
+
+    @Override
+    public Map<String, GarbageCollectorStats> garbageCollectors() {
+        Set<String> garbageCollectors = getMetricNames(REGISTRY_METRICSET_GARBAGE_COLLECTORS);
+        Map<String, GarbageCollectorStats> gcStats = new HashMap<>();
+        for (String garbageCollector : garbageCollectors) {
+            gcStats.put(garbageCollector, new GarbageCollectorStats(
+                    (Long) getMetric(REGISTRY_METRICSET_GARBAGE_COLLECTORS + "." + garbageCollector + ".count"),
+                    (Long) getMetric(REGISTRY_METRICSET_GARBAGE_COLLECTORS + "." + garbageCollector + ".time")));
+        }
+        return Collections.unmodifiableMap(gcStats);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Set<String> deadlockedThreads() {
+        return (Set<String>) getMetric(THREADS_DEADLOCKS);
+    }
+
+    @Override
+    public Map<Thread.State, Double> threadStatePercentages() {
+        int totalThreadCount = (Integer) getMetric(THREADS_COUNT);
+        final Map<Thread.State, Double> threadStatePercentages = new HashMap<Thread.State, Double>();
+        for (Thread.State state : Thread.State.values()) {
+            threadStatePercentages.put(state, (Integer) getMetric(REGISTRY_METRICSET_THREADS + "." + state.name().toLowerCase() + ".count") / (double) totalThreadCount);
+        }
+        return Collections.unmodifiableMap(threadStatePercentages);
+    }
+
+    @Override
+    public void threadDump(OutputStream out) {
+        throw new UnsupportedOperationException("This operation has not yet been implemented");
+    }
+
+    @Override
+    public Map<String, BufferPoolStats> getBufferPoolStats() {
+        throw new UnsupportedOperationException("This operation has not yet been implemented");
+    }
+}
\ No newline at end of file
diff --git a/nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java b/nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
new file mode 100644
index 0000000..3ffdd41
--- /dev/null
+++ b/nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
@@ -0,0 +1,280 @@
+/*
+ * 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.metrics.jvm;
+
+import org.apache.nifi.processor.DataUnit;
+
+import java.io.OutputStream;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Provides methods for retrieving metrics from the JVM.
+ */
+public interface JvmMetrics {
+
+    /**
+     * Returns the total initial memory of the current JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return total Heap and non-heap initial JVM memory in the given {@link DataUnit}
+     */
+    double totalInit(DataUnit dataUnit);
+
+    /**
+     * Returns the total memory currently used by the current JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return total Heap and non-heap memory currently used by JVM in the given {@link DataUnit}
+     */
+    double totalUsed(DataUnit dataUnit);
+
+    /**
+     * Returns the total memory currently used by the current JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return total Heap and non-heap memory currently used by JVM in the given {@link DataUnit}
+     */
+    double totalMax(DataUnit dataUnit);
+
+    /**
+     * Returns the total memory committed to the JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return total Heap and non-heap memory currently committed to the JVM in the given {@link DataUnit}
+     */
+    double totalCommitted(DataUnit dataUnit);
+
+    /**
+     * Returns the heap initial memory of the current JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return Heap initial JVM memory in the given {@link DataUnit}
+     */
+    double heapInit(DataUnit dataUnit);
+
+    /**
+     * Returns the heap memory currently used by the current JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return Heap memory currently used by JVM in the given {@link DataUnit}
+     */
+    double heapUsed(DataUnit dataUnit);
+
+    /**
+     * Returns the heap memory currently used by the current JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return Heap memory currently used by JVM in the given {@link DataUnit}
+     */
+    double heapMax(DataUnit dataUnit);
+
+    /**
+     * Returns the heap memory committed to the JVM.
+     *
+     * @param dataUnit The {@link DataUnit} to which the metric will be converted
+     *
+     * @return Heap memory currently committed to the JVM in the given {@link DataUnit}
+     */
+    double heapCommitted(DataUnit dataUnit);
+
+    /**
+     * Returns the percentage of the JVM's heap which is being used.
+     *
+     * @return the percentage of the JVM's heap which is being used
+     */
+    double heapUsage();
+
+    /**
+     * Returns the percentage of the JVM's non-heap memory (e.g., direct buffers) which is being
+     * used.
+     *
+     * @return the percentage of the JVM's non-heap memory which is being used
+     */
+    double nonHeapUsage();
+
+    /**
+     * Returns a map of memory pool names to the percentage of that pool which is being used.
+     *
+     * @return a map of memory pool names to the percentage of that pool which is being used
+     */
+    Map<String, Double> memoryPoolUsage();
+
+    /**
+     * Returns the percentage of available file descriptors which are currently in use.
+     *
+     * @return the percentage of available file descriptors which are currently in use, or {@code
+     *         NaN} if the running JVM does not have access to this information
+     */
+    double fileDescriptorUsage();
+
+    /**
+     * Returns the version of the currently-running jvm.
+     *
+     * @return the version of the currently-running jvm, eg "1.6.0_24"
+     * @see <a href="http://java.sun.com/j2se/versioning_naming.html">J2SE SDK/JRE Version String
+     *      Naming Convention</a>
+     */
+    String version();
+
+    /**
+     * Returns the name of the currently-running jvm.
+     *
+     * @return the name of the currently-running jvm, eg  "Java HotSpot(TM) Client VM"
+     * @see <a href="http://download.oracle.com/javase/6/docs/api/java/lang/System.html#getProperties()">System.getProperties()</a>
+     */
+    String name();
+
+    /**
+     * Returns the number of seconds the JVM process has been running.
+     *
+     * @return the number of seconds the JVM process has been running
+     */
+    long uptime();
+
+    /**
+     * Returns the number of live threads (includes {@link #daemonThreadCount()}.
+     *
+     * @return the number of live threads
+     */
+    int threadCount();
+
+    /**
+     * Returns the number of live daemon threads.
+     *
+     * @return the number of live daemon threads
+     */
+    int daemonThreadCount();
+
+    /**
+     * Returns a map of garbage collector names to garbage collector information.
+     *
+     * @return a map of garbage collector names to garbage collector information
+     */
+    Map<String, GarbageCollectorStats> garbageCollectors();
+
+    /**
+     * Returns a set of strings describing deadlocked threads, if any are deadlocked.
+     *
+     * @return a set of any deadlocked threads
+     */
+    Set<String> deadlockedThreads();
+
+    /**
+     * Returns a map of thread states to the percentage of all threads which are in that state.
+     *
+     * @return a map of thread states to percentages
+     */
+    Map<Thread.State, Double> threadStatePercentages();
+
+    /**
+     * Dumps all of the threads' current information to an output stream.
+     *
+     * @param out an output stream
+     */
+    void threadDump(OutputStream out);
+
+    Map<String, BufferPoolStats> getBufferPoolStats();
+
+    /**
+     * Per-GC statistics.
+     */
+    class GarbageCollectorStats {
+        private final long runs, timeMS;
+
+        GarbageCollectorStats(long runs, long timeMS) {
+            this.runs = runs;
+            this.timeMS = timeMS;
+        }
+
+        /**
+         * Returns the number of times the garbage collector has run.
+         *
+         * @return the number of times the garbage collector has run
+         */
+        public long getRuns() {
+            return runs;
+        }
+
+        /**
+         * Returns the amount of time in the given unit the garbage collector has taken in total.
+         *
+         * @param unit    the time unit for the return value
+         * @return the amount of time in the given unit the garbage collector
+         */
+        public long getTime(TimeUnit unit) {
+            return unit.convert(timeMS, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    /**
+     * The management interface for a buffer pool, for example a pool of {@link
+     * java.nio.ByteBuffer#allocateDirect direct} or {@link java.nio.MappedByteBuffer mapped}
+     * buffers.
+     */
+    class BufferPoolStats {
+        private final long count, memoryUsed, totalCapacity;
+
+        BufferPoolStats(long count, long memoryUsed, long totalCapacity) {
+            this.count = count;
+            this.memoryUsed = memoryUsed;
+            this.totalCapacity = totalCapacity;
+        }
+
+        /**
+         * Returns an estimate of the number of buffers in the pool.
+         *
+         * @return An estimate of the number of buffers in this pool
+         */
+        public long getCount() {
+            return count;
+        }
+
+        /**
+         * Returns an estimate of the memory that the Java virtual machine is using for this buffer
+         * pool. The value returned by this method may differ from the estimate of the total {@link
+         * #getTotalCapacity capacity} of the buffers in this pool. This difference is explained by
+         * alignment, memory allocator, and other implementation specific reasons.
+         *
+         * @return An estimate of the memory that the Java virtual machine is using for this buffer
+         *         pool in bytes, or {@code -1L} if an estimate of the memory usage is not
+         *         available
+         */
+        public long getMemoryUsed(DataUnit dataUnit) {
+            return (long)dataUnit.convert(memoryUsed, DataUnit.B);
+        }
+
+        /**
+         * Returns an estimate of the total capacity of the buffers in this pool. A buffer's
+         * capacity is the number of elements it contains and the value returned by this method is
+         * an estimate of the total capacity of buffers in the pool in bytes.
+         *
+         * @return An estimate of the total capacity of the buffers in this pool in bytes
+         */
+        public long getTotalCapacity(DataUnit dataUnit) {
+            return totalCapacity;
+        }
+    }
+}
\ No newline at end of file
diff --git a/nifi-commons/nifi-metrics/src/test/groovy/org/apache/nifi/metrics/jvm/JmxJvmMetricsSpec.groovy b/nifi-commons/nifi-metrics/src/test/groovy/org/apache/nifi/metrics/jvm/JmxJvmMetricsSpec.groovy
new file mode 100644
index 0000000..f503cf5
--- /dev/null
+++ b/nifi-commons/nifi-metrics/src/test/groovy/org/apache/nifi/metrics/jvm/JmxJvmMetricsSpec.groovy
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.metrics.jvm
+
+import org.apache.nifi.processor.DataUnit
+import spock.lang.Specification
+import spock.lang.Unroll
+
+class JmxJvmMetricsSpec extends Specification {
+
+    @Unroll
+    def "Get numeric metric for #metricName via method #methodName"() {
+        given:
+        def jmxJvmMetrics = JmxJvmMetrics.instance
+
+        when:
+        def metricResult = jmxJvmMetrics."$methodName"(dataUnit).doubleValue()
+
+        then:
+        noExceptionThrown()
+        metricResult != -1
+
+
+        where:
+        metricName               | methodName       | dataUnit
+        "memory.total.init"      | "totalInit"      | DataUnit.B
+        "memory.total.used"      | "totalUsed"      | DataUnit.B
+        "memory.total.max"       | "totalMax"       | DataUnit.B
+        "memory.total.committed" | "totalCommitted" | DataUnit.B
+        "memory.heap.init"       | "heapInit"       | DataUnit.B
+        "memory.heap.used"       | "heapUsed"       | DataUnit.B
+        "memory.heap.max"        | "heapMax"        | DataUnit.B
+        "memory.heap.committed"  | "heapCommitted"  | DataUnit.B
+        "memory.total.init"      | "totalInit"      | DataUnit.B
+        "memory.total.init"      | "totalInit"      | DataUnit.B
+        "memory.total.init"      | "totalInit"      | DataUnit.B
+    }
+
+    @Unroll
+    def "Get percentage metric for #metricName via method #methodName"() {
+        given:
+        def jmxJvmMetrics = JmxJvmMetrics.instance
+
+        when:
+        def metricResult = jmxJvmMetrics."$methodName"()
+
+        then:
+        noExceptionThrown()
+        metricResult instanceof Double
+        metricResult != 0.0
+
+        where:
+        metricName                | methodName
+        "memory.heap.usage"       | "heapUsage"
+        "memory.non-heap.usage"   | "nonHeapUsage"
+        "os.filedescriptor.usage" | "fileDescriptorUsage"
+    }
+
+    def "Memory pool metric names exist"() {
+        given:
+        def jmxJvmMetrics = JmxJvmMetrics.instance
+
+        when:
+        def names = jmxJvmMetrics.getMetricNames(JmxJvmMetrics.REGISTRY_METRICSET_MEMORY + ".pools")
+
+        then:
+        names.size() > 0
+    }
+
+    @Unroll
+    def "Get string map metric for #metricName via method #methodName"() {
+        given:
+        def jmxJvmMetrics = JmxJvmMetrics.instance
+
+        when:
+        def metricResult = jmxJvmMetrics."$methodName"()
+
+        then:
+        noExceptionThrown()
+        metricResult.keySet().size() > 0
+
+        where:
+        metricName           | methodName
+        "memory.pools.usage" | "memoryPoolUsage"
+        "garbage-collectors" | "garbageCollectors"
+    }
+}
diff --git a/nifi-commons/pom.xml b/nifi-commons/pom.xml
index 59bfc0b..9441d8d 100644
--- a/nifi-commons/pom.xml
+++ b/nifi-commons/pom.xml
@@ -39,5 +39,6 @@
         <module>nifi-schema-utils</module>
     	<module>nifi-record</module>
         <module>nifi-record-path</module>
+        <module>nifi-metrics</module>
     </modules>
 </project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/src/main/resources/META-INF/NOTICE
index 0e1a4b1..8f5b87b 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-nar/src/main/resources/META-INF/NOTICE
@@ -10,20 +10,25 @@ Apache Software License v2
 
 The following binary components are provided under the Apache Software License v2
 
-  (ASLv2) Yammer Metrics
+  (ASLv2) Dropwizard Metrics
     The following NOTICE information applies:
       Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
+      Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
       This product includes software developed by Coda Hale and Yammer, Inc.
 
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
 
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
           http://creativecommons.org/publicdomain/zero/1.0/
 
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
+
 ************************
 Common Development and Distribution License 1.1
 ************************
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml
index 3685283..1a5b4f8 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/pom.xml
@@ -48,6 +48,11 @@
             <artifactId>nifi-reporting-utils</artifactId>
             <version>1.10.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-metrics</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
         <!-- test dependencies -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
index 0568b3e..886e891 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/main/java/org/apache/nifi/reporting/ambari/AmbariReportingTask.java
@@ -16,8 +16,6 @@
  */
 package org.apache.nifi.reporting.ambari;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
-
 import org.apache.nifi.annotation.configuration.DefaultSchedule;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -26,6 +24,8 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
+import org.apache.nifi.metrics.jvm.JvmMetrics;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.ReportingContext;
@@ -95,7 +95,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
 
     private volatile Client client;
     private volatile JsonBuilderFactory factory;
-    private volatile VirtualMachineMetrics virtualMachineMetrics;
+    private volatile JvmMetrics virtualMachineMetrics;
     private volatile JsonObject previousMetrics = null;
 
     private final MetricsService metricsService = new MetricsService();
@@ -115,7 +115,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
         final Map<String, ?> config = Collections.emptyMap();
         factory = Json.createBuilderFactory(config);
         client = createClient();
-        virtualMachineMetrics = VirtualMachineMetrics.getInstance();
+        virtualMachineMetrics = JmxJvmMetrics.getInstance();
         previousMetrics = null;
     }
 
diff --git a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/test/java/org/apache/nifi/reporting/ambari/metrics/TestMetricsService.java b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/test/java/org/apache/nifi/reporting/ambari/metrics/TestMetricsService.java
index ec0cf6e..32f6881 100644
--- a/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/test/java/org/apache/nifi/reporting/ambari/metrics/TestMetricsService.java
+++ b/nifi-nar-bundles/nifi-ambari-bundle/nifi-ambari-reporting-task/src/test/java/org/apache/nifi/reporting/ambari/metrics/TestMetricsService.java
@@ -16,9 +16,10 @@
  */
 package org.apache.nifi.reporting.ambari.metrics;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
+import org.apache.nifi.metrics.jvm.JvmMetrics;
 import org.apache.nifi.reporting.util.metrics.MetricNames;
 import org.apache.nifi.reporting.util.metrics.MetricsService;
 import org.junit.Assert;
@@ -117,7 +118,7 @@ public class TestMetricsService {
 
     @Test
     public void testGetVirtualMachineMetrics() {
-        final VirtualMachineMetrics virtualMachineMetrics = VirtualMachineMetrics.getInstance();
+        final JvmMetrics virtualMachineMetrics = JmxJvmMetrics.getInstance();
         final MetricsService service = new MetricsService();
 
         final Map<String,String> metrics = service.getMetrics(virtualMachineMetrics);
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/src/main/resources/META-INF/NOTICE
index 6616bee..a3ac493 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-nar/src/main/resources/META-INF/NOTICE
@@ -463,20 +463,25 @@ The following binary components are provided under the Apache Software License v
       "GCC RUntime Library Exception"
       http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
 
-  (ASLv2) Yammer Metrics
+  (ASLv2) Dropwizard Metrics
     The following NOTICE information applies:
       Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
+      Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
       This product includes software developed by Coda Hale and Yammer, Inc.
 
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
 
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
           http://creativecommons.org/publicdomain/zero/1.0/
 
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
+
   (ASLv2) ZkClient
     The following NOTICE information applies:
       ZkClient
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/src/main/resources/META-INF/NOTICE
index e5b0b5e..f3e3b62 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-nar/src/main/resources/META-INF/NOTICE
@@ -93,6 +93,15 @@ The following binary components are provided under the Apache Software License v
     The following NOTICE information applies:
 	Copyright (c) 2010-2013 Coda Hale, Yammer.com
 
+    This product includes software developed by Coda Hale and Yammer, Inc.
+
+    This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+    LongAdder), which was released with the following comments:
+
+        Written by Doug Lea with assistance from members of JCP JSR-166
+        Expert Group and released to the public domain, as explained at
+        http://creativecommons.org/publicdomain/zero/1.0/
+
   (ASLv2) The Netty Project
       The following NOTICE information applies:
   	Copyright 2014 The Netty Project
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/src/main/resources/META-INF/NOTICE
index c998953..7653831 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-api-nar/src/main/resources/META-INF/NOTICE
@@ -1,226 +1,235 @@
-nifi-cassandra-services-api-nar
-Copyright 2016-2019 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-******************
-Apache Software License v2
-******************
-
-  (ASLv2) The Netty Project
-        The following NOTICE information applies:
-    	Copyright 2014 The Netty Project
-    	-------------------------------------------------------------------------------
-    	This product contains the extensions to Java Collections Framework which has
-    	been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
-    	  * LICENSE:
-    	    * license/LICENSE.jsr166y.txt (Public Domain)
-    	  * HOMEPAGE:
-    	    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
-    	    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-    	This product contains a modified version of Robert Harder's Public Domain
-    	Base64 Encoder and Decoder, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.base64.txt (Public Domain)
-    	  * HOMEPAGE:
-    	    * http://iharder.sourceforge.net/current/java/base64/
-
-    	This product contains a modified portion of 'Webbit', an event based
-    	WebSocket and HTTP server, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.webbit.txt (BSD License)
-    	  * HOMEPAGE:
-    	    * https://github.com/joewalnes/webbit
-
-    	This product contains a modified portion of 'SLF4J', a simple logging
-    	facade for Java, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.slf4j.txt (MIT License)
-    	  * HOMEPAGE:
-    	    * http://www.slf4j.org/
-
-    	This product contains a modified portion of 'Apache Harmony', an open source
-    	Java SE, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.harmony.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * http://archive.apache.org/dist/harmony/
-
-    	This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-    	and decompression library written by Matthew J. Francis. It can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.jbzip2.txt (MIT License)
-    	  * HOMEPAGE:
-    	    * https://code.google.com/p/jbzip2/
-
-    	This product contains a modified portion of 'libdivsufsort', a C API library to construct
-    	the suffix array and the Burrows-Wheeler transformed string for any input string of
-    	a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.libdivsufsort.txt (MIT License)
-    	  * HOMEPAGE:
-    	    * https://github.com/y-256/libdivsufsort
-
-    	This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
-    	 which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.jctools.txt (ASL2 License)
-    	  * HOMEPAGE:
-    	    * https://github.com/JCTools/JCTools
-
-    	This product optionally depends on 'JZlib', a re-implementation of zlib in
-    	pure Java, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.jzlib.txt (BSD style License)
-    	  * HOMEPAGE:
-    	    * http://www.jcraft.com/jzlib/
-
-    	This product optionally depends on 'Compress-LZF', a Java library for encoding and
-    	decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * https://github.com/ning/compress
-
-    	This product optionally depends on 'lz4', a LZ4 Java compression
-    	and decompression library written by Adrien Grand. It can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.lz4.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * https://github.com/jpountz/lz4-java
-
-    	This product optionally depends on 'lzma-java', a LZMA Java compression
-    	and decompression library, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.lzma-java.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * https://github.com/jponge/lzma-java
-
-    	This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-    	and decompression library written by William Kinney. It can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.jfastlz.txt (MIT License)
-    	  * HOMEPAGE:
-    	    * https://code.google.com/p/jfastlz/
-
-    	This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-    	interchange format, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.protobuf.txt (New BSD License)
-    	  * HOMEPAGE:
-    	    * https://github.com/google/protobuf
-
-    	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/
-
-    	This product optionally depends on 'Snappy', a compression library produced
-    	by Google Inc, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.snappy.txt (New BSD License)
-    	  * HOMEPAGE:
-    	    * https://github.com/google/snappy
-
-    	This product optionally depends on 'JBoss Marshalling', an alternative Java
-    	serialization API, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
-    	  * HOMEPAGE:
-    	    * http://www.jboss.org/jbossmarshalling
-
-    	This product optionally depends on 'Caliper', Google's micro-
-    	benchmarking framework, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.caliper.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * https://github.com/google/caliper
-
-    	This product optionally depends on 'Apache Commons Logging', a logging
-    	framework, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.commons-logging.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * http://commons.apache.org/logging/
-
-    	This product optionally depends on 'Apache Log4J', a logging framework, which
-    	can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.log4j.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * http://logging.apache.org/log4j/
-
-    	This product optionally depends on 'Aalto XML', an ultra-high performance
-    	non-blocking XML processor, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * http://wiki.fasterxml.com/AaltoHome
-
-    	This product contains a modified version of 'HPACK', a Java implementation of
-    	the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.hpack.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-    	    * https://github.com/twitter/hpack
-
-    	This product contains a modified portion of 'Apache Commons Lang', a Java library
-    	provides utilities for the java.lang API, which can be obtained at:
-
-    	  * LICENSE:
-    	    * license/LICENSE.commons-lang.txt (Apache License 2.0)
-    	  * HOMEPAGE:
-        * https://commons.apache.org/proper/commons-lang/
-
-        This product contains a forked and modified version of Tomcat Native
-
-          * LICENSE:
-            * ASL2
-          * HOMEPAGE:
-            * http://tomcat.apache.org/native-doc/
-            * https://svn.apache.org/repos/asf/tomcat/native/
-
-  (ASLv2) Guava
-    The following NOTICE information applies:
-      Guava
-      Copyright 2015 The Guava Authors
-
-  (ASLv2) Dropwizard Metrics
-    The following NOTICE information applies:
-	Copyright (c) 2010-2013 Coda Hale, Yammer.com
-
-	************************
-    Eclipse Public License 1.0
-    ************************
-
-    The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
-
-        (EPL 2.0)(GPL 2)(LGPL 2.1) JNR Posix ( jnr.posix ) https://github.com/jnr/jnr-posix/blob/master/LICENSE.txt
-
+nifi-cassandra-services-api-nar
+Copyright 2016-2019 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+  (ASLv2) The Netty Project
+        The following NOTICE information applies:
+    	Copyright 2014 The Netty Project
+    	-------------------------------------------------------------------------------
+    	This product contains the extensions to Java Collections Framework which has
+    	been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jsr166y.txt (Public Domain)
+    	  * HOMEPAGE:
+    	    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    	    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+    	This product contains a modified version of Robert Harder's Public Domain
+    	Base64 Encoder and Decoder, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.base64.txt (Public Domain)
+    	  * HOMEPAGE:
+    	    * http://iharder.sourceforge.net/current/java/base64/
+
+    	This product contains a modified portion of 'Webbit', an event based
+    	WebSocket and HTTP server, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.webbit.txt (BSD License)
+    	  * HOMEPAGE:
+    	    * https://github.com/joewalnes/webbit
+
+    	This product contains a modified portion of 'SLF4J', a simple logging
+    	facade for Java, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.slf4j.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * http://www.slf4j.org/
+
+    	This product contains a modified portion of 'Apache Harmony', an open source
+    	Java SE, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.harmony.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://archive.apache.org/dist/harmony/
+
+    	This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+    	and decompression library written by Matthew J. Francis. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jbzip2.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * https://code.google.com/p/jbzip2/
+
+    	This product contains a modified portion of 'libdivsufsort', a C API library to construct
+    	the suffix array and the Burrows-Wheeler transformed string for any input string of
+    	a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.libdivsufsort.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * https://github.com/y-256/libdivsufsort
+
+    	This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+    	 which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jctools.txt (ASL2 License)
+    	  * HOMEPAGE:
+    	    * https://github.com/JCTools/JCTools
+
+    	This product optionally depends on 'JZlib', a re-implementation of zlib in
+    	pure Java, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jzlib.txt (BSD style License)
+    	  * HOMEPAGE:
+    	    * http://www.jcraft.com/jzlib/
+
+    	This product optionally depends on 'Compress-LZF', a Java library for encoding and
+    	decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/ning/compress
+
+    	This product optionally depends on 'lz4', a LZ4 Java compression
+    	and decompression library written by Adrien Grand. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.lz4.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/jpountz/lz4-java
+
+    	This product optionally depends on 'lzma-java', a LZMA Java compression
+    	and decompression library, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/jponge/lzma-java
+
+    	This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+    	and decompression library written by William Kinney. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jfastlz.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * https://code.google.com/p/jfastlz/
+
+    	This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+    	interchange format, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.protobuf.txt (New BSD License)
+    	  * HOMEPAGE:
+    	    * https://github.com/google/protobuf
+
+    	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/
+
+    	This product optionally depends on 'Snappy', a compression library produced
+    	by Google Inc, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.snappy.txt (New BSD License)
+    	  * HOMEPAGE:
+    	    * https://github.com/google/snappy
+
+    	This product optionally depends on 'JBoss Marshalling', an alternative Java
+    	serialization API, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+    	  * HOMEPAGE:
+    	    * http://www.jboss.org/jbossmarshalling
+
+    	This product optionally depends on 'Caliper', Google's micro-
+    	benchmarking framework, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.caliper.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/google/caliper
+
+    	This product optionally depends on 'Apache Commons Logging', a logging
+    	framework, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://commons.apache.org/logging/
+
+    	This product optionally depends on 'Apache Log4J', a logging framework, which
+    	can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.log4j.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://logging.apache.org/log4j/
+
+    	This product optionally depends on 'Aalto XML', an ultra-high performance
+    	non-blocking XML processor, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://wiki.fasterxml.com/AaltoHome
+
+    	This product contains a modified version of 'HPACK', a Java implementation of
+    	the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.hpack.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/twitter/hpack
+
+    	This product contains a modified portion of 'Apache Commons Lang', a Java library
+    	provides utilities for the java.lang API, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+        * https://commons.apache.org/proper/commons-lang/
+
+        This product contains a forked and modified version of Tomcat Native
+
+          * LICENSE:
+            * ASL2
+          * HOMEPAGE:
+            * http://tomcat.apache.org/native-doc/
+            * https://svn.apache.org/repos/asf/tomcat/native/
+
+  (ASLv2) Guava
+    The following NOTICE information applies:
+      Guava
+      Copyright 2015 The Guava Authors
+
+  (ASLv2) Dropwizard Metrics
+    The following NOTICE information applies:
+	Copyright (c) 2010-2013 Coda Hale, Yammer.com
+
+    This product includes software developed by Coda Hale and Yammer, Inc.
+
+    This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+    LongAdder), which was released with the following comments:
+
+        Written by Doug Lea with assistance from members of JCP JSR-166
+        Expert Group and released to the public domain, as explained at
+        http://creativecommons.org/publicdomain/zero/1.0/
+
+	************************
+    Eclipse Public License 1.0
+    ************************
+
+    The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
+
+        (EPL 2.0)(GPL 2)(LGPL 2.1) JNR Posix ( jnr.posix ) https://github.com/jnr/jnr-posix/blob/master/LICENSE.txt
+
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/src/main/resources/META-INF/NOTICE
index 12d44a0..72acf75 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services-nar/src/main/resources/META-INF/NOTICE
@@ -1,291 +1,300 @@
-nifi-cassandra-services-nar
-Copyright 2016-2019 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-******************
-Apache Software License v2
-******************
-
-The following binary components are provided under the Apache Software License v2
-
-  (ASLv2) DataStax Java Driver for Apache Cassandra - Core
-      The following NOTICE information applies:
-        DataStax Java Driver for Apache Cassandra - Core
-        Copyright (C) 2012-2017 DataStax Inc.
-
-  (ASLv2) Jackson JSON processor
-    The following NOTICE information applies:
-      # Jackson JSON processor
-
-      Jackson is a high-performance, Free/Open Source JSON processing library.
-      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
-      been in development since 2007.
-      It is currently developed by a community of developers, as well as supported
-      commercially by FasterXML.com.
-
-      ## Licensing
-
-      Jackson core and extension components may licensed under different licenses.
-      To find the details that apply to this artifact see the accompanying LICENSE file.
-      For more information, including possible other licensing options, contact
-      FasterXML.com (http://fasterxml.com).
-
-      ## Credits
-
-      A list of contributors may be found from CREDITS file, which is included
-      in some artifacts (usually source distributions); but is always available
-      from the source code management (SCM) system project uses.
-
-  (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())
-
-  (ASLv2) Guava
-    The following NOTICE information applies:
-      Guava
-      Copyright 2015 The Guava Authors
-
-  (ASLv2) JSON-SMART
-    The following NOTICE information applies:
-    Copyright 2011 JSON-SMART authors
-
-  (ASLv2) Dropwizard Metrics
-    The following NOTICE information applies:
-	Copyright (c) 2010-2013 Coda Hale, Yammer.com
-
-  (ASLv2) The Netty Project
-      The following NOTICE information applies:
-  	Copyright 2014 The Netty Project
-  	-------------------------------------------------------------------------------
-  	This product contains the extensions to Java Collections Framework which has
-  	been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
-  	  * LICENSE:
-  	    * license/LICENSE.jsr166y.txt (Public Domain)
-  	  * HOMEPAGE:
-  	    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
-  	    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-  	This product contains a modified version of Robert Harder's Public Domain
-  	Base64 Encoder and Decoder, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.base64.txt (Public Domain)
-  	  * HOMEPAGE:
-  	    * http://iharder.sourceforge.net/current/java/base64/
-
-  	This product contains a modified portion of 'Webbit', an event based
-  	WebSocket and HTTP server, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.webbit.txt (BSD License)
-  	  * HOMEPAGE:
-  	    * https://github.com/joewalnes/webbit
-
-  	This product contains a modified portion of 'SLF4J', a simple logging
-  	facade for Java, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.slf4j.txt (MIT License)
-  	  * HOMEPAGE:
-  	    * http://www.slf4j.org/
-
-  	This product contains a modified portion of 'Apache Harmony', an open source
-  	Java SE, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.harmony.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * http://archive.apache.org/dist/harmony/
-
-  	This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-  	and decompression library written by Matthew J. Francis. It can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.jbzip2.txt (MIT License)
-  	  * HOMEPAGE:
-  	    * https://code.google.com/p/jbzip2/
-
-  	This product contains a modified portion of 'libdivsufsort', a C API library to construct
-  	the suffix array and the Burrows-Wheeler transformed string for any input string of
-  	a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.libdivsufsort.txt (MIT License)
-  	  * HOMEPAGE:
-  	    * https://github.com/y-256/libdivsufsort
-
-  	This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
-  	 which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.jctools.txt (ASL2 License)
-  	  * HOMEPAGE:
-  	    * https://github.com/JCTools/JCTools
-
-  	This product optionally depends on 'JZlib', a re-implementation of zlib in
-  	pure Java, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.jzlib.txt (BSD style License)
-  	  * HOMEPAGE:
-  	    * http://www.jcraft.com/jzlib/
-
-  	This product optionally depends on 'Compress-LZF', a Java library for encoding and
-  	decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * https://github.com/ning/compress
-
-  	This product optionally depends on 'lz4', a LZ4 Java compression
-  	and decompression library written by Adrien Grand. It can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.lz4.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * https://github.com/jpountz/lz4-java
-
-  	This product optionally depends on 'lzma-java', a LZMA Java compression
-  	and decompression library, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.lzma-java.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * https://github.com/jponge/lzma-java
-
-  	This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-  	and decompression library written by William Kinney. It can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.jfastlz.txt (MIT License)
-  	  * HOMEPAGE:
-  	    * https://code.google.com/p/jfastlz/
-
-  	This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-  	interchange format, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.protobuf.txt (New BSD License)
-  	  * HOMEPAGE:
-  	    * https://github.com/google/protobuf
-
-  	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/
-
-  	This product optionally depends on 'Snappy', a compression library produced
-  	by Google Inc, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.snappy.txt (New BSD License)
-  	  * HOMEPAGE:
-  	    * https://github.com/google/snappy
-
-  	This product optionally depends on 'JBoss Marshalling', an alternative Java
-  	serialization API, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
-  	  * HOMEPAGE:
-  	    * http://www.jboss.org/jbossmarshalling
-
-  	This product optionally depends on 'Caliper', Google's micro-
-  	benchmarking framework, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.caliper.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * https://github.com/google/caliper
-
-  	This product optionally depends on 'Apache Commons Logging', a logging
-  	framework, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.commons-logging.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * http://commons.apache.org/logging/
-
-  	This product optionally depends on 'Apache Log4J', a logging framework, which
-  	can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.log4j.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * http://logging.apache.org/log4j/
-
-  	This product optionally depends on 'Aalto XML', an ultra-high performance
-  	non-blocking XML processor, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * http://wiki.fasterxml.com/AaltoHome
-
-  	This product contains a modified version of 'HPACK', a Java implementation of
-  	the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.hpack.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-  	    * https://github.com/twitter/hpack
-
-  	This product contains a modified portion of 'Apache Commons Lang', a Java library
-  	provides utilities for the java.lang API, which can be obtained at:
-
-  	  * LICENSE:
-  	    * license/LICENSE.commons-lang.txt (Apache License 2.0)
-  	  * HOMEPAGE:
-      * https://commons.apache.org/proper/commons-lang/
-
-      This product contains a forked and modified version of Tomcat Native
-
-        * LICENSE:
-          * ASL2
-        * HOMEPAGE:
-          * http://tomcat.apache.org/native-doc/
-          * https://svn.apache.org/repos/asf/tomcat/native/
-
-  (ASLv2) Objenesis
-      The following NOTICE information applies:
-        Objenesis
-        Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita
-
-************************
-Eclipse Public License 1.0
-************************
-
-The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
-
-    (EPL 2.0)(GPL 2)(LGPL 2.1) JNR Posix ( jnr.posix ) https://github.com/jnr/jnr-posix/blob/master/LICENSE.txt
-    (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.2.3 - http://logback.qos.ch/)
-    (EPL 1.0)(LGPL 2.1) Logback Core (ch.qos.logback:logback-core:jar:1.2.3 - http://logback.qos.ch/)
+nifi-cassandra-services-nar
+Copyright 2016-2019 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) DataStax Java Driver for Apache Cassandra - Core
+      The following NOTICE information applies:
+        DataStax Java Driver for Apache Cassandra - Core
+        Copyright (C) 2012-2017 DataStax Inc.
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (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())
+
+  (ASLv2) Guava
+    The following NOTICE information applies:
+      Guava
+      Copyright 2015 The Guava Authors
+
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+    Copyright 2011 JSON-SMART authors
+
+  (ASLv2) Dropwizard Metrics
+    The following NOTICE information applies:
+	Copyright (c) 2010-2013 Coda Hale, Yammer.com
+
+    This product includes software developed by Coda Hale and Yammer, Inc.
+
+    This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+    LongAdder), which was released with the following comments:
+
+        Written by Doug Lea with assistance from members of JCP JSR-166
+        Expert Group and released to the public domain, as explained at
+        http://creativecommons.org/publicdomain/zero/1.0/
+
+  (ASLv2) The Netty Project
+      The following NOTICE information applies:
+  	Copyright 2014 The Netty Project
+  	-------------------------------------------------------------------------------
+  	This product contains the extensions to Java Collections Framework which has
+  	been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  	  * LICENSE:
+  	    * license/LICENSE.jsr166y.txt (Public Domain)
+  	  * HOMEPAGE:
+  	    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+  	    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+  	This product contains a modified version of Robert Harder's Public Domain
+  	Base64 Encoder and Decoder, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.base64.txt (Public Domain)
+  	  * HOMEPAGE:
+  	    * http://iharder.sourceforge.net/current/java/base64/
+
+  	This product contains a modified portion of 'Webbit', an event based
+  	WebSocket and HTTP server, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.webbit.txt (BSD License)
+  	  * HOMEPAGE:
+  	    * https://github.com/joewalnes/webbit
+
+  	This product contains a modified portion of 'SLF4J', a simple logging
+  	facade for Java, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.slf4j.txt (MIT License)
+  	  * HOMEPAGE:
+  	    * http://www.slf4j.org/
+
+  	This product contains a modified portion of 'Apache Harmony', an open source
+  	Java SE, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.harmony.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * http://archive.apache.org/dist/harmony/
+
+  	This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+  	and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.jbzip2.txt (MIT License)
+  	  * HOMEPAGE:
+  	    * https://code.google.com/p/jbzip2/
+
+  	This product contains a modified portion of 'libdivsufsort', a C API library to construct
+  	the suffix array and the Burrows-Wheeler transformed string for any input string of
+  	a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.libdivsufsort.txt (MIT License)
+  	  * HOMEPAGE:
+  	    * https://github.com/y-256/libdivsufsort
+
+  	This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+  	 which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.jctools.txt (ASL2 License)
+  	  * HOMEPAGE:
+  	    * https://github.com/JCTools/JCTools
+
+  	This product optionally depends on 'JZlib', a re-implementation of zlib in
+  	pure Java, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.jzlib.txt (BSD style License)
+  	  * HOMEPAGE:
+  	    * http://www.jcraft.com/jzlib/
+
+  	This product optionally depends on 'Compress-LZF', a Java library for encoding and
+  	decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * https://github.com/ning/compress
+
+  	This product optionally depends on 'lz4', a LZ4 Java compression
+  	and decompression library written by Adrien Grand. It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.lz4.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * https://github.com/jpountz/lz4-java
+
+  	This product optionally depends on 'lzma-java', a LZMA Java compression
+  	and decompression library, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * https://github.com/jponge/lzma-java
+
+  	This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+  	and decompression library written by William Kinney. It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.jfastlz.txt (MIT License)
+  	  * HOMEPAGE:
+  	    * https://code.google.com/p/jfastlz/
+
+  	This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+  	interchange format, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.protobuf.txt (New BSD License)
+  	  * HOMEPAGE:
+  	    * https://github.com/google/protobuf
+
+  	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/
+
+  	This product optionally depends on 'Snappy', a compression library produced
+  	by Google Inc, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.snappy.txt (New BSD License)
+  	  * HOMEPAGE:
+  	    * https://github.com/google/snappy
+
+  	This product optionally depends on 'JBoss Marshalling', an alternative Java
+  	serialization API, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  	  * HOMEPAGE:
+  	    * http://www.jboss.org/jbossmarshalling
+
+  	This product optionally depends on 'Caliper', Google's micro-
+  	benchmarking framework, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.caliper.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * https://github.com/google/caliper
+
+  	This product optionally depends on 'Apache Commons Logging', a logging
+  	framework, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * http://commons.apache.org/logging/
+
+  	This product optionally depends on 'Apache Log4J', a logging framework, which
+  	can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.log4j.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * http://logging.apache.org/log4j/
+
+  	This product optionally depends on 'Aalto XML', an ultra-high performance
+  	non-blocking XML processor, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * http://wiki.fasterxml.com/AaltoHome
+
+  	This product contains a modified version of 'HPACK', a Java implementation of
+  	the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.hpack.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+  	    * https://github.com/twitter/hpack
+
+  	This product contains a modified portion of 'Apache Commons Lang', a Java library
+  	provides utilities for the java.lang API, which can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  	  * HOMEPAGE:
+      * https://commons.apache.org/proper/commons-lang/
+
+      This product contains a forked and modified version of Tomcat Native
+
+        * LICENSE:
+          * ASL2
+        * HOMEPAGE:
+          * http://tomcat.apache.org/native-doc/
+          * https://svn.apache.org/repos/asf/tomcat/native/
+
+  (ASLv2) Objenesis
+      The following NOTICE information applies:
+        Objenesis
+        Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita
+
+************************
+Eclipse Public License 1.0
+************************
+
+The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
+
+    (EPL 2.0)(GPL 2)(LGPL 2.1) JNR Posix ( jnr.posix ) https://github.com/jnr/jnr-posix/blob/master/LICENSE.txt
+    (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.2.3 - http://logback.qos.ch/)
+    (EPL 1.0)(LGPL 2.1) Logback Core (ch.qos.logback:logback-core:jar:1.2.3 - http://logback.qos.ch/)
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/src/main/resources/META-INF/NOTICE
index d71a09f..7f0240f 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-nar/src/main/resources/META-INF/NOTICE
@@ -10,20 +10,25 @@ Apache Software License v2
 
 The following binary components are provided under the Apache Software License v2
 
-  (ASLv2) Metrics
+  (ASLv2) Dropwizard Metrics
     The following NOTICE information applies:
       Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
+      Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
       This product includes software developed by Coda Hale and Yammer, Inc.
 
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
 
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
           http://creativecommons.org/publicdomain/zero/1.0/
 
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
+
   (ASLv2) Apache Commons Lang
     The following NOTICE information applies:
       Apache Commons Lang
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml
index 727b74e..90f4eaf 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/pom.xml
@@ -40,23 +40,18 @@
             <version>1.0</version>
         </dependency>
         <dependency>
-            <groupId>com.yammer.metrics</groupId>
-            <artifactId>metrics-core</artifactId>
-            <version>2.2.0</version>
-        </dependency>
-        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
+            <artifactId>nifi-metrics</artifactId>
             <version>1.10.0-SNAPSHOT</version>
         </dependency>
         <dependency>
-            <groupId>io.dropwizard.metrics</groupId>
-            <artifactId>metrics-core</artifactId>
-            <version>3.1.0</version>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.coursera</groupId>
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
index bba9658..1c0704e 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/DataDogReportingTask.java
@@ -22,7 +22,6 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.AtomicDouble;
-import com.yammer.metrics.core.VirtualMachineMetrics;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
@@ -34,6 +33,7 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.ReportingContext;
@@ -102,7 +102,7 @@ public class DataDogReportingTask extends AbstractReportingTask {
     private String statusId;
     private ConcurrentHashMap<String, AtomicDouble> metricsMap;
     private Map<String, String> defaultTags;
-    private volatile VirtualMachineMetrics virtualMachineMetrics;
+    private volatile JmxJvmMetrics virtualMachineMetrics;
     private Logger logger = LoggerFactory.getLogger(getClass().getName());
 
     @OnScheduled
@@ -113,7 +113,7 @@ public class DataDogReportingTask extends AbstractReportingTask {
         metricsMap = getMetricsMap();
         metricsPrefix = METRICS_PREFIX.getDefaultValue();
         environment = ENVIRONMENT.getDefaultValue();
-        virtualMachineMetrics = VirtualMachineMetrics.getInstance();
+        virtualMachineMetrics = JmxJvmMetrics.getInstance();
         ddMetricRegistryBuilder.setMetricRegistry(metricRegistry)
                 .setTags(metricsService.getAllTagsList());
     }
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/metrics/MetricsService.java b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/metrics/MetricsService.java
index d1419eb..3e26073 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/metrics/MetricsService.java
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/main/java/org/apache/nifi/reporting/datadog/metrics/MetricsService.java
@@ -16,11 +16,12 @@
  */
 package org.apache.nifi.reporting.datadog.metrics;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
 import org.apache.nifi.controller.status.ConnectionStatus;
 import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
+import org.apache.nifi.processor.DataUnit;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -126,10 +127,10 @@ public class MetricsService {
     }
 
     //virtual machine metrics
-    public Map<String, Double> getJVMMetrics(VirtualMachineMetrics virtualMachineMetrics) {
+    public Map<String, Double> getJVMMetrics(JmxJvmMetrics virtualMachineMetrics) {
         final Map<String, Double> metrics = new HashMap<>();
         metrics.put(MetricNames.JVM_UPTIME, new Double(virtualMachineMetrics.uptime()));
-        metrics.put(MetricNames.JVM_HEAP_USED, new Double(virtualMachineMetrics.heapUsed()));
+        metrics.put(MetricNames.JVM_HEAP_USED, new Double(virtualMachineMetrics.heapUsed(DataUnit.B)));
         metrics.put(MetricNames.JVM_HEAP_USAGE, new Double(virtualMachineMetrics.heapUsage()));
         metrics.put(MetricNames.JVM_NON_HEAP_USAGE, new Double(virtualMachineMetrics.nonHeapUsage()));
         metrics.put(MetricNames.JVM_THREAD_COUNT, new Double(virtualMachineMetrics.threadCount()));
@@ -156,7 +157,7 @@ public class MetricsService {
             }
         }
 
-        for (Map.Entry<String, VirtualMachineMetrics.GarbageCollectorStats> entry : virtualMachineMetrics.garbageCollectors().entrySet()) {
+        for (Map.Entry<String, JmxJvmMetrics.GarbageCollectorStats> entry : virtualMachineMetrics.garbageCollectors().entrySet()) {
             final String gcName = entry.getKey().replace(" ", "");
             final long runs = entry.getValue().getRuns();
             final long timeMS = entry.getValue().getTime(TimeUnit.MILLISECONDS);
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestDataDogReportingTask.java b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestDataDogReportingTask.java
index e26223b..2b5de42 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestDataDogReportingTask.java
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestDataDogReportingTask.java
@@ -21,10 +21,10 @@ import com.codahale.metrics.MetricRegistry;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.AtomicDouble;
-import com.yammer.metrics.core.VirtualMachineMetrics;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
 import org.apache.nifi.reporting.EventAccess;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.reporting.ReportingContext;
@@ -60,7 +60,7 @@ public class TestDataDogReportingTask {
     private ReportingContext context;
     private ReportingInitializationContext initContext;
     private ConfigurationContext configurationContext;
-    private volatile VirtualMachineMetrics virtualMachineMetrics;
+    private volatile JmxJvmMetrics virtualMachineMetrics;
     private Logger logger;
 
     @Before
@@ -92,7 +92,7 @@ public class TestDataDogReportingTask {
         //Mockito.when(initContext.getLogger()).thenReturn(logger);
         metricsMap = new ConcurrentHashMap<>();
         metricRegistry = Mockito.mock(MetricRegistry.class);
-        virtualMachineMetrics = VirtualMachineMetrics.getInstance();
+        virtualMachineMetrics = JmxJvmMetrics.getInstance();
         metricsService = Mockito.mock(MetricsService.class);
 
     }
@@ -106,7 +106,7 @@ public class TestDataDogReportingTask {
         dataDogReportingTask.onTrigger(context);
 
         verify(metricsService, atLeast(1)).getProcessorMetrics(Mockito.<ProcessorStatus>any());
-        verify(metricsService, atLeast(1)).getJVMMetrics(Mockito.<VirtualMachineMetrics>any());
+        verify(metricsService, atLeast(1)).getJVMMetrics(Mockito.<JmxJvmMetrics>any());
     }
 
 
diff --git a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestMetricsService.java b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestMetricsService.java
index 2cf5648..5f3c31e 100644
--- a/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestMetricsService.java
+++ b/nifi-nar-bundles/nifi-datadog-bundle/nifi-datadog-reporting-task/src/test/java/org/apache/nifi/reporting/datadog/TestMetricsService.java
@@ -16,9 +16,9 @@
  */
 package org.apache.nifi.reporting.datadog;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
 import org.apache.nifi.reporting.datadog.metrics.MetricNames;
 import org.apache.nifi.reporting.datadog.metrics.MetricsService;
 import org.junit.Assert;
@@ -91,7 +91,7 @@ public class TestMetricsService {
     //test JVM status metric retrieving
     @Test
     public void testGetVirtualMachineMetrics() {
-        final VirtualMachineMetrics virtualMachineMetrics = VirtualMachineMetrics.getInstance();
+        final JmxJvmMetrics virtualMachineMetrics = JmxJvmMetrics.getInstance();
 
         final Map<String, Double> metrics = metricsService.getJVMMetrics(virtualMachineMetrics);
         Assert.assertTrue(metrics.containsKey(MetricNames.JVM_UPTIME));
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml
index 7f206bb..e260482 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/pom.xml
@@ -41,15 +41,16 @@
             <version>3.8.1</version>
         </dependency>
         <dependency>
-            <groupId>com.yammer.metrics</groupId>
-            <artifactId>metrics-core</artifactId>
-            <version>2.2.0</version>
-        </dependency>
-        <dependency>
             <groupId>org.glassfish</groupId>
             <artifactId>javax.json</artifactId>
             <version>1.0.4</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-metrics</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
 
 </project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/metrics/MetricsService.java b/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/metrics/MetricsService.java
index ed3922a..593d329 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/metrics/MetricsService.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/metrics/MetricsService.java
@@ -26,10 +26,10 @@ import javax.json.JsonObjectBuilder;
 
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
+import org.apache.nifi.metrics.jvm.JvmMetrics;
+import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.reporting.util.metrics.api.MetricFields;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
-
 /**
  * A service used to produce key/value metrics based on a given input.
  */
@@ -90,7 +90,7 @@ public class MetricsService {
      * @param virtualMachineMetrics a VirtualMachineMetrics instance to get metrics from
      * @return a map of metrics from the given VirtualMachineStatus
      */
-    public Map<String,String> getMetrics(VirtualMachineMetrics virtualMachineMetrics) {
+    public Map<String,String> getMetrics(JvmMetrics virtualMachineMetrics) {
         final Map<String,String> metrics = new HashMap<>();
 
         Map<String,Integer> integerMetrics = getIntegerMetrics(virtualMachineMetrics);
@@ -135,20 +135,20 @@ public class MetricsService {
         }
     }
 
-    private Map<String,Double> getDoubleMetrics(VirtualMachineMetrics virtualMachineMetrics) {
+    private Map<String,Double> getDoubleMetrics(JvmMetrics virtualMachineMetrics) {
         final Map<String,Double> metrics = new HashMap<>();
-        metrics.put(MetricNames.JVM_HEAP_USED, virtualMachineMetrics.heapUsed());
+        metrics.put(MetricNames.JVM_HEAP_USED, virtualMachineMetrics.heapUsed(DataUnit.B));
         metrics.put(MetricNames.JVM_HEAP_USAGE, virtualMachineMetrics.heapUsage());
         metrics.put(MetricNames.JVM_NON_HEAP_USAGE, virtualMachineMetrics.nonHeapUsage());
         metrics.put(MetricNames.JVM_FILE_DESCRIPTOR_USAGE, virtualMachineMetrics.fileDescriptorUsage());
         return metrics;
     }
 
-    private Map<String,Long> getLongMetrics(VirtualMachineMetrics virtualMachineMetrics) {
+    private Map<String,Long> getLongMetrics(JvmMetrics virtualMachineMetrics) {
         final Map<String,Long> metrics = new HashMap<>();
         metrics.put(MetricNames.JVM_UPTIME, virtualMachineMetrics.uptime());
 
-        for (Map.Entry<String,VirtualMachineMetrics.GarbageCollectorStats> entry : virtualMachineMetrics.garbageCollectors().entrySet()) {
+        for (Map.Entry<String,JvmMetrics.GarbageCollectorStats> entry : virtualMachineMetrics.garbageCollectors().entrySet()) {
             final String gcName = entry.getKey().replace(" ", "");
             final long runs = entry.getValue().getRuns();
             final long timeMS = entry.getValue().getTime(TimeUnit.MILLISECONDS);
@@ -159,7 +159,7 @@ public class MetricsService {
         return metrics;
     }
 
-    private Map<String,Integer> getIntegerMetrics(VirtualMachineMetrics virtualMachineMetrics) {
+    private Map<String,Integer> getIntegerMetrics(JvmMetrics virtualMachineMetrics) {
         final Map<String,Integer> metrics = new HashMap<>();
         metrics.put(MetricNames.JVM_DAEMON_THREAD_COUNT, virtualMachineMetrics.daemonThreadCount());
         metrics.put(MetricNames.JVM_THREAD_COUNT, virtualMachineMetrics.threadCount());
@@ -187,7 +187,7 @@ public class MetricsService {
         return metrics;
     }
 
-    public JsonObject getMetrics(JsonBuilderFactory factory, ProcessGroupStatus status, VirtualMachineMetrics virtualMachineMetrics,
+    public JsonObject getMetrics(JsonBuilderFactory factory, ProcessGroupStatus status, JvmMetrics virtualMachineMetrics,
             String applicationId, String id, String hostname, long currentTimeMillis, int availableProcessors, double systemLoad) {
         JsonObjectBuilder objectBuilder = factory.createObjectBuilder()
                 .add(MetricFields.APP_ID, applicationId)
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml b/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml
index 3323a83..3ab2757 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/pom.xml
@@ -32,7 +32,6 @@
     </modules>
 
     <properties>
-        <yammer.metrics.version>2.2.0</yammer.metrics.version>
         <jolt.version>0.1.1</jolt.version>
         <jersey.version>2.26</jersey.version>
     </properties>
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/NOTICE
index 6c6ecb5..34b98eb 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-nifi-kafka-nar
+nifi-kafka-0-10-nar
 Copyright 2014-2019 The Apache Software Foundation
 
 This product includes software developed at
@@ -23,20 +23,6 @@ The following binary components are provided under the Apache Software License v
       Apache Kafka
       Copyright 2012 The Apache Software Foundation.
 
-  (ASLv2) Yammer Metrics
-    The following NOTICE information applies:
-      Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-      This product includes software developed by Coda Hale and Yammer, Inc.
-
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
-
-          Written by Doug Lea with assistance from members of JCP JSR-166
-          Expert Group and released to the public domain, as explained at
-          http://creativecommons.org/publicdomain/zero/1.0/
-
   (ASLv2) Snappy Java
     The following NOTICE information applies:
       This product includes software developed by Google
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
index b38c4fc..0baaed0 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-nifi-kafka-nar
+nifi-kafka-0-11-nar
 Copyright 2014-2019 The Apache Software Foundation
 
 This product includes software developed at
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/NOTICE
index a307a87..8d93d9a 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-nifi-kafka-nar
+nifi-kafka-0-8-nar
 Copyright 2014-2019 The Apache Software Foundation
 
 This product includes software developed at
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE
index 0e7a2bc..11e8acf 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE
@@ -23,20 +23,6 @@ The following binary components are provided under the Apache Software License v
       Apache Kafka
       Copyright 2012 The Apache Software Foundation.
 
-  (ASLv2) Yammer Metrics
-    The following NOTICE information applies:
-      Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-      This product includes software developed by Coda Hale and Yammer, Inc.
-
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
-
-          Written by Doug Lea with assistance from members of JCP JSR-166
-          Expert Group and released to the public domain, as explained at
-          http://creativecommons.org/publicdomain/zero/1.0/
-
   (ASLv2) Snappy Java
     The following NOTICE information applies:
       This product includes software developed by Google
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE
index b38c4fc..8ecc087 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-nifi-kafka-nar
+nifi-kafka-1-0-nar
 Copyright 2014-2019 The Apache Software Foundation
 
 This product includes software developed at
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..43a2a3b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,233 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+  The binary distribution of this product bundles 'Bouncy Castle JDK 1.5'
+  under an MIT style license.
+
+    Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/src/main/resources/META-INF/NOTICE
similarity index 99%
copy from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
copy to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/src/main/resources/META-INF/NOTICE
index b38c4fc..8ecc087 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-0-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-nifi-kafka-nar
+nifi-kafka-1-0-nar
 Copyright 2014-2019 The Apache Software Foundation
 
 This product includes software developed at
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/src/main/resources/META-INF/NOTICE
index 9f90b1b..abe3198 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api-nar/src/main/resources/META-INF/NOTICE
@@ -14,7 +14,6 @@ The following binary components are provided under the Apache Software License v
     The following NOTICE information applies:
       Metrics
       Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
-
       This product includes software developed by Coda Hale and Yammer, Inc.
 
       This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
@@ -22,4 +21,10 @@ The following binary components are provided under the Apache Software License v
 
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
-      http://creativecommons.org/publicdomain/zero/1.0/
\ No newline at end of file
+          http://creativecommons.org/publicdomain/zero/1.0/
+
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml
index 5bc74cb..c5be492 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporter-service-api/pom.xml
@@ -22,4 +22,11 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>nifi-metrics-reporter-service-api</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-metrics</artifactId>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/src/main/resources/META-INF/NOTICE
index 0b4acb6..a7985b0 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-nar/src/main/resources/META-INF/NOTICE
@@ -14,7 +14,6 @@ The following binary components are provided under the Apache Software License v
     The following NOTICE information applies:
       Metrics
       Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
-
       This product includes software developed by Coda Hale and Yammer, Inc.
 
       This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
@@ -22,4 +21,10 @@ The following binary components are provided under the Apache Software License v
 
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
-      http://creativecommons.org/publicdomain/zero/1.0/
\ No newline at end of file
+          http://creativecommons.org/publicdomain/zero/1.0/
+
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml
index 76c2bc5..80f4cd1 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/pom.xml
@@ -38,12 +38,11 @@
         <dependency>
             <groupId>io.dropwizard.metrics</groupId>
             <artifactId>metrics-graphite</artifactId>
-            <version>3.1.2</version>
+            <version>4.1.0</version>
         </dependency>
         <dependency>
-            <groupId>io.dropwizard.metrics</groupId>
-            <artifactId>metrics-jvm</artifactId>
-            <version>3.1.2</version>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-metrics</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/test/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterServiceTest.java b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/test/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterServiceTest.java
index 30ab5db..e7755c1 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/test/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterServiceTest.java
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/nifi-metrics-reporting-task/src/test/java/org/apache/nifi/metrics/reporting/reporter/service/GraphiteMetricReporterServiceTest.java
@@ -35,7 +35,6 @@ import java.nio.charset.StandardCharsets;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
 
 /**
  * Test class for {@link GraphiteMetricReporterService}.
@@ -143,7 +142,6 @@ public class GraphiteMetricReporterServiceTest {
         testedService = new TestableGraphiteMetricReporterService();
         runner.addControllerService(SERVICE_IDENTIFIER, testedService);
         setServiceProperties(TEST_HOST, TEST_PORT, TEST_CHARSET, METRIC_NAMES_PREFIX);
-        when(graphiteSenderMock.isConnected()).thenReturn(false);
         runner.enableControllerService(testedService);
 
         ScheduledReporter createdReporter = testedService.createReporter(metricRegistryStub);
diff --git a/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml b/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml
index 92a6698..f8522b7 100644
--- a/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-metrics-reporting-bundle/pom.xml
@@ -34,6 +34,11 @@
         <dependencies>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-metrics</artifactId>
+                <version>1.10.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-metrics-reporting-task</artifactId>
                 <version>1.10.0-SNAPSHOT</version>
             </dependency>
@@ -42,11 +47,6 @@
 
     <dependencies>
         <dependency>
-            <groupId>io.dropwizard.metrics</groupId>
-            <artifactId>metrics-core</artifactId>
-            <version>3.1.2</version>
-        </dependency>
-        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
             <scope>provided</scope>
diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/src/main/resources/META-INF/NOTICE
index ec59289..55ad405 100644
--- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/src/main/resources/META-INF/NOTICE
@@ -9,20 +9,25 @@ Apache Software License v2
 
 The following binary components are provided under the Apache Software License v2
 
-  (ASLv2) Yammer Metrics
+  (ASLv2) Dropwizard Metrics
     The following NOTICE information applies:
       Metrics
-      Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
+      Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
       This product includes software developed by Coda Hale and Yammer, Inc.
 
-      This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-      with the following comments:
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
 
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
           http://creativecommons.org/publicdomain/zero/1.0/
 
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
+
   (ASLv2) Prometheus Simple Client libraries
     Copyright 2012-2019 The Prometheus Authors
 
diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml
index 780d522..8290222 100644
--- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml
+++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml
@@ -20,7 +20,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/ma
 </parent>
 <properties>
         <prometheus.version>0.3.0</prometheus.version>
-        <yammer.metrics.version>2.2.0</yammer.metrics.version>
 </properties>
 
 <artifactId>nifi-prometheus-reporting-task</artifactId>
@@ -28,9 +27,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/ma
 
 <dependencies>
         <dependency>
-                <groupId>com.yammer.metrics</groupId>
-                <artifactId>metrics-core</artifactId>
-                <version>${yammer.metrics.version}</version>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-metrics</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
         </dependency>
         <dependency>
                 <groupId>org.apache.nifi</groupId>
@@ -54,7 +53,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/ma
                <version>${prometheus.version}</version>
         </dependency>
         <!-- Exposition servlet -->
-       <dependency>
+        <dependency>
                 <groupId>io.prometheus</groupId>
                 <artifactId>simpleclient_servlet</artifactId>
                 <version>${prometheus.version}</version>
diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java
index 76ef126..521d9ca 100644
--- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java
+++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/PrometheusServer.java
@@ -30,6 +30,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
 import org.apache.nifi.reporting.ReportingContext;
 import org.apache.nifi.reporting.prometheus.api.PrometheusMetricsUtil;
 import org.apache.nifi.ssl.SSLContextService;
@@ -44,8 +45,6 @@ import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
-
 import io.prometheus.client.CollectorRegistry;
 import io.prometheus.client.exporter.common.TextFormat;
 
@@ -76,7 +75,7 @@ public class PrometheusServer {
             TextFormat.write004(osw, nifiRegistry.metricFamilySamples());
 
             if (PrometheusServer.this.sendJvmMetrics == true) {
-                jvmRegistry = PrometheusMetricsUtil.createJvmMetrics(VirtualMachineMetrics.getInstance(), PrometheusServer.this.instanceId);
+                jvmRegistry = PrometheusMetricsUtil.createJvmMetrics(JmxJvmMetrics.getInstance(), PrometheusServer.this.instanceId);
                 TextFormat.write004(osw, jvmRegistry.metricFamilySamples());
             }
 
diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java
index f4fa642..6227260 100644
--- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java
+++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java
@@ -25,12 +25,12 @@ import org.apache.nifi.controller.status.PortStatus;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.controller.status.ProcessorStatus;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
-
 import io.prometheus.client.CollectorRegistry;
 import io.prometheus.client.Gauge;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 import org.apache.nifi.controller.status.TransmissionStatus;
+import org.apache.nifi.metrics.jvm.JvmMetrics;
+import org.apache.nifi.processor.DataUnit;
 
 public class PrometheusMetricsUtil {
 
@@ -398,8 +398,8 @@ public class PrometheusMetricsUtil {
         return NIFI_REGISTRY;
     }
 
-    public static CollectorRegistry createJvmMetrics(VirtualMachineMetrics jvmMetrics, String instanceId) {
-        JVM_HEAP_USED.labels(instanceId).set(jvmMetrics.heapUsed());
+    public static CollectorRegistry createJvmMetrics(JvmMetrics jvmMetrics, String instanceId) {
+        JVM_HEAP_USED.labels(instanceId).set(jvmMetrics.heapUsed(DataUnit.B));
         JVM_HEAP_USAGE.labels(instanceId).set(jvmMetrics.heapUsage());
         JVM_HEAP_NON_USAGE.labels(instanceId).set(jvmMetrics.nonHeapUsage());
 
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE
index 8c3d16b..7bba730 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE
@@ -41,19 +41,24 @@ The following binary components are provided under the Apache Software License v
          This product includes software developed at
          The Apache Software Foundation (http://www.apache.org/).
 
-  (ASLv2) Yammer Metrics
+  (ASLv2) Dropwizard Metrics
       The following NOTICE information applies:
-        Metrics
-        Copyright 2010-2012 Coda Hale and Yammer, Inc.
-
-        This product includes software developed by Coda Hale and Yammer, Inc.
-
-        This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
-        with the following comments:
-
-            Written by Doug Lea with assistance from members of JCP JSR-166
-            Expert Group and released to the public domain, as explained at
-            http://creativecommons.org/publicdomain/zero/1.0/
+          Metrics
+          Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
+          This product includes software developed by Coda Hale and Yammer, Inc.
+
+          This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+          LongAdder), which was released with the following comments:
+
+              Written by Doug Lea with assistance from members of JCP JSR-166
+              Expert Group and released to the public domain, as explained at
+              http://creativecommons.org/publicdomain/zero/1.0/
+
+          The derived work in the nifi-metrics module is adapted from
+            https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+          and can be found in
+            nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+            nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
 
   (ASLv2) Apache Ivy
       The following NOTICE information applies:
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
index 94f6ec2..9e20aee 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
@@ -32,6 +32,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-metrics</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
         </dependency>
         <dependency>
@@ -91,11 +95,6 @@
             <artifactId>commons-net</artifactId>
             <version>3.6</version>
         </dependency>
-        <dependency>
-            <groupId>com.yammer.metrics</groupId>
-            <artifactId>metrics-core</artifactId>
-            <version>2.2.0</version>
-        </dependency>
         <!-- For Jython 2.7.1 -->
         <dependency>
             <groupId>xerces</groupId>
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
index fb1d737..189412e 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/reporting/script/ScriptedReportingTask.java
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.reporting.script;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.Restricted;
@@ -31,6 +30,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.script.ScriptEngineConfigurator;
@@ -75,7 +75,7 @@ public class ScriptedReportingTask extends AbstractReportingTask {
 
     protected volatile ScriptingComponentHelper scriptingComponentHelper = new ScriptingComponentHelper();
     private volatile String scriptToRun = null;
-    private volatile VirtualMachineMetrics vmMetrics;
+    private volatile JmxJvmMetrics vmMetrics;
 
     /**
      * Returns a list of property descriptors supported by this processor. The list always includes properties such as
@@ -143,7 +143,7 @@ public class ScriptedReportingTask extends AbstractReportingTask {
             throw new ProcessException(ioe);
         }
 
-        vmMetrics = VirtualMachineMetrics.getInstance();
+        vmMetrics = JmxJvmMetrics.getInstance();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/pom.xml
index 8bb7861..e3f9da6 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-scripting-bundle/pom.xml
@@ -62,6 +62,12 @@
                 <scope>provided</scope>
             </dependency>
             <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-metrics</artifactId>
+                <version>1.10.0-SNAPSHOT</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
                 <groupId>org.codehaus.groovy</groupId>
                 <artifactId>groovy-all</artifactId>
                 <version>2.4.15</version>
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-nar/src/main/resources/META-INF/NOTICE
index 75c1551..e668bd1 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-nar/src/main/resources/META-INF/NOTICE
@@ -33,6 +33,25 @@ The following binary components are provided under the Apache Software License v
       in some artifacts (usually source distributions); but is always available
       from the source code management (SCM) system project uses.
 
+  (ASLv2) Dropwizard Metrics
+    The following NOTICE information applies:
+      Metrics
+      Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team
+      This product includes software developed by Coda Hale and Yammer, Inc.
+
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
+
+          Written by Doug Lea with assistance from members of JCP JSR-166
+          Expert Group and released to the public domain, as explained at
+          http://creativecommons.org/publicdomain/zero/1.0/
+
+      The derived work in the nifi-metrics module is adapted from
+        https://github.com/dropwizard/metrics/blob/v2.2.0/metrics-core/src/main/java/com/yammer/metrics/core/VirtualMachineMetrics.java
+      and can be found in
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JvmMetrics.java
+        nifi-commons/nifi-metrics/src/main/java/org/apache/nifi/metrics/jvm/JmxJvmMetrics.java
+
 ************************
 Common Development and Distribution License 1.1
 ************************
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteMetricsReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteMetricsReportingTask.java
index e17c2c8..1053433 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteMetricsReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteMetricsReportingTask.java
@@ -48,6 +48,8 @@ import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.metrics.jvm.JmxJvmMetrics;
+import org.apache.nifi.metrics.jvm.JvmMetrics;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.remote.Transaction;
@@ -56,8 +58,6 @@ import org.apache.nifi.reporting.util.metrics.MetricNames;
 import org.apache.nifi.reporting.util.metrics.MetricsService;
 import org.apache.nifi.reporting.util.metrics.api.MetricsBuilder;
 
-import com.yammer.metrics.core.VirtualMachineMetrics;
-
 @Tags({"status", "metrics", "site", "site to site"})
 @CapabilityDescription("Publishes same metrics as the Ambari Reporting task using the Site To Site protocol.")
 public class SiteToSiteMetricsReportingTask extends AbstractSiteToSiteReportingTask {
@@ -150,7 +150,7 @@ public class SiteToSiteMetricsReportingTask extends AbstractSiteToSiteReportingT
             return;
         }
 
-        final VirtualMachineMetrics virtualMachineMetrics = VirtualMachineMetrics.getInstance();
+        final JvmMetrics virtualMachineMetrics = JmxJvmMetrics.getInstance();
         final Map<String, ?> config = Collections.emptyMap();
         final JsonBuilderFactory factory = Json.createBuilderFactory(config);
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service-nar/src/main/resources/META-INF/NOTICE
index dd7ee77..00dbf2f 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service-nar/src/main/resources/META-INF/NOTICE
@@ -314,3 +314,14 @@ Apache Software License v2
        Jetty Web Container
        Copyright 1995-2019 Mort Bay Consulting Pty Ltd.
 
+  (ASLv2) Dropwizard Metrics
+      Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+      This product includes software developed by Coda Hale and Yammer, Inc.
+
+      This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+      LongAdder), which was released with the following comments:
+
+          Written by Doug Lea with assistance from members of JCP JSR-166
+          Expert Group and released to the public domain, as explained at
+          http://creativecommons.org/publicdomain/zero/1.0/
\ No newline at end of file