You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sa...@apache.org on 2022/10/31 20:39:00 UTC

[ignite-3] branch main updated: IGNITE-17357 JMX metrics exporter (#1234)

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

sanpwc pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new ef308321b5 IGNITE-17357 JMX metrics exporter (#1234)
ef308321b5 is described below

commit ef308321b53047ef0deef421881db868624772f2
Author: Kirill Gusakov <kg...@gmail.com>
AuthorDate: Mon Oct 31 23:38:54 2022 +0300

    IGNITE-17357 JMX metrics exporter (#1234)
---
 .../apache/ignite/internal/util/IgniteUtils.java   |  19 ++
 .../metrics/exporters/TestPullMetricExporter.java  |  13 +-
 .../metrics/exporters/TestPushMetricExporter.java  |  14 +-
 .../ignite/internal/metrics/MetricManager.java     |  28 ++-
 .../metrics/exporters/BasicMetricExporter.java     |  18 +-
 .../internal/metrics/exporters/MetricExporter.java |  35 +++-
 .../metrics/exporters/PushMetricExporter.java      |   5 +-
 .../JmxExporterConfigurationSchema.java            |   6 +-
 .../metrics/exporters/jmx/JmxExporter.java         | 165 ++++++++++++++++
 .../metrics/exporters/jmx/MetricSetMbean.java      | 181 ++++++++++++++++++
 .../ignite/internal/metrics/JmxExporterTest.java   | 211 +++++++++++++++++++++
 .../internal/metrics/exporters/TestExporter.java   |  22 ++-
 12 files changed, 682 insertions(+), 35 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 20f2f36a7a..93d032a976 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -52,6 +52,8 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
 import java.util.stream.Stream;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.util.worker.IgniteWorker;
 import org.apache.ignite.lang.IgniteInternalException;
@@ -103,6 +105,11 @@ public class IgniteUtils {
     /** Class cache. */
     private static final ConcurrentMap<ClassLoader, ConcurrentMap<String, Class<?>>> classCache = new ConcurrentHashMap<>();
 
+    /**
+     * Root package for JMX MBeans.
+     */
+    private static final String JMX_MBEAN_PACKAGE = "org.apache";
+
     /**
      * Get JDK version.
      *
@@ -862,4 +869,16 @@ public class IgniteUtils {
 
         return result;
     }
+
+    /**
+     * Produce new MBean name according to received group and name.
+     *
+     * @param group pkg:group=value part of MBean name.
+     * @param name pkg:name=value part of MBean name.
+     * @return new ObjectName.
+     * @throws MalformedObjectNameException if MBean name can't be formed from the received arguments.
+     */
+    public static ObjectName makeMbeanName(String group, String name) throws MalformedObjectNameException {
+        return new ObjectName(String.format("%s:group=%s,name=%s", JMX_MBEAN_PACKAGE, group, name));
+    }
 }
diff --git a/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricExporter.java b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricExporter.java
index 2d2da80e18..d0bb8c2f95 100644
--- a/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricExporter.java
+++ b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricExporter.java
@@ -23,6 +23,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import org.apache.ignite.internal.metrics.Metric;
+import org.apache.ignite.internal.metrics.MetricProvider;
 import org.apache.ignite.internal.metrics.MetricSet;
 
 /**
@@ -52,7 +53,9 @@ public class TestPullMetricExporter extends BasicMetricExporter<TestPullMetricsE
     }
 
     @Override
-    public void start() {
+    public void start(MetricProvider metricProvider, TestPullMetricsExporterView conf) {
+        super.start(metricProvider, conf);
+
         executorService.execute(() -> {
             while (true) {
                 waitForRequest();
@@ -93,6 +96,14 @@ public class TestPullMetricExporter extends BasicMetricExporter<TestPullMetricsE
         return EXPORTER_NAME;
     }
 
+    @Override
+    public void addMetricSet(MetricSet metricSet) {
+    }
+
+    @Override
+    public void removeMetricSet(String metricSetName) {
+    }
+
     private void waitForRequest() {
         synchronized (obj) {
             try {
diff --git a/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricExporter.java b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricExporter.java
index 668ff987e5..2a990a206b 100644
--- a/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricExporter.java
+++ b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricExporter.java
@@ -35,10 +35,10 @@ public class TestPushMetricExporter extends PushMetricExporter<TestPushMetricsEx
     private long period;
 
     @Override
-    public void init(MetricProvider metricsProvider, TestPushMetricsExporterView configuration) {
-        super.init(metricsProvider, configuration);
-
+    public void start(MetricProvider metricsProvider, TestPushMetricsExporterView configuration) {
         period = configuration.period();
+
+        super.start(metricsProvider, configuration);
     }
 
     public static void setOutputStream(OutputStream outputStream) {
@@ -84,4 +84,12 @@ public class TestPushMetricExporter extends PushMetricExporter<TestPushMetricsEx
     public String name() {
         return EXPORTER_NAME;
     }
+
+    @Override
+    public void addMetricSet(MetricSet metricSet) {
+    }
+
+    @Override
+    public void removeMetricSet(String metricSetName) {
+    }
 }
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/MetricManager.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/MetricManager.java
index 01b603b283..bb491208dd 100644
--- a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/MetricManager.java
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/MetricManager.java
@@ -107,6 +107,8 @@ public class MetricManager implements IgniteComponent {
         for (MetricExporter metricExporter : enabledMetricExporters.values()) {
             metricExporter.stop();
         }
+
+        enabledMetricExporters.clear();
     }
 
     /**
@@ -143,7 +145,13 @@ public class MetricManager implements IgniteComponent {
      * @return Metric set, or {@code null} if already enabled.
      */
     public MetricSet enable(MetricSource src) {
-        return registry.enable(src);
+        MetricSet enabled = registry.enable(src);
+
+        if (enabled != null) {
+            enabledMetricExporters.values().forEach(e -> e.addMetricSet(enabled));
+        }
+
+        return enabled;
     }
 
     /**
@@ -153,7 +161,13 @@ public class MetricManager implements IgniteComponent {
      * @return Metric set, or {@code null} if already enabled.
      */
     public MetricSet enable(final String srcName) {
-        return registry.enable(srcName);
+        MetricSet enabled = registry.enable(srcName);
+
+        if (enabled != null) {
+            enabledMetricExporters.values().forEach(e -> e.addMetricSet(enabled));
+        }
+
+        return enabled;
     }
 
     /**
@@ -178,15 +192,19 @@ public class MetricManager implements IgniteComponent {
      */
     public void disable(MetricSource src) {
         registry.disable(src);
+
+        enabledMetricExporters.values().forEach(e -> e.removeMetricSet(src.name()));
     }
 
     /**
      * Disable metric source by name. See {@link MetricRegistry#disable(String)}.
      *
-     * @param srcName Source name.
+     * @param srcName Metric source name.
      */
     public void disable(final String srcName) {
         registry.disable(srcName);
+
+        enabledMetricExporters.values().forEach(e -> e.removeMetricSet(srcName));
     }
 
     /**
@@ -215,9 +233,7 @@ public class MetricManager implements IgniteComponent {
         MetricExporter<T> exporter = availableExporters.get(exporterName);
 
         if (exporter != null) {
-            exporter.init(metricsProvider, exporterConfiguration);
-
-            exporter.start();
+            exporter.start(metricsProvider, exporterConfiguration);
 
             enabledMetricExporters.put(exporter.name(), exporter);
         } else {
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java
index 85ef05cd22..b61e89ef32 100644
--- a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java
@@ -31,19 +31,29 @@ public abstract class BasicMetricExporter<CfgT extends ExporterView> implements
     /** Metrics provider. */
     private MetricProvider metricsProvider;
 
-    protected CfgT configuration;
+    /** Exporter's configuration view. */
+    private CfgT configuration;
 
     /** {@inheritDoc} */
     @Override
-    public void init(MetricProvider metricsProvider, CfgT configuration) {
+    public void start(MetricProvider metricsProvider, CfgT configuration) {
         this.metricsProvider = metricsProvider;
         this.configuration = configuration;
     }
 
     /** {@inheritDoc} */
     @Override
-    public void reconfigure(@Nullable CfgT newValue) {
-        configuration = newValue;
+    public synchronized void reconfigure(@Nullable CfgT newVal) {
+        configuration = newVal;
+    }
+
+    /**
+     * Returns current exporter configuration.
+     *
+     * @return Current exporter configuration
+     */
+    protected synchronized CfgT configuration() {
+        return configuration;
     }
 
     /**
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/MetricExporter.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/MetricExporter.java
index 55759fd4d2..2bae2dfcbf 100644
--- a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/MetricExporter.java
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/MetricExporter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.metrics.exporters;
 
 import org.apache.ignite.internal.metrics.MetricProvider;
+import org.apache.ignite.internal.metrics.MetricSet;
 import org.apache.ignite.internal.metrics.exporters.configuration.ExporterConfiguration;
 import org.apache.ignite.internal.metrics.exporters.configuration.ExporterView;
 
@@ -32,18 +33,13 @@ import org.apache.ignite.internal.metrics.exporters.configuration.ExporterView;
  * Pull exporters should extend {@link BasicMetricExporter}.
  */
 public interface MetricExporter<CfgT extends ExporterView> {
-    /**
-     * Initialize metric exporter with the provider of available metrics.
-     *
-     * @param metricProvider Metrics provider
-     * @param configuration CfgT
-     */
-    void init(MetricProvider metricProvider, CfgT configuration);
-
     /**
      * Start metrics exporter. Here all needed listeners, schedulers etc. should be started.
+     *
+     * @param metricProvider Provider of metric sources.
+     * @param configuration Exporter configuration view.
      */
-    void start();
+    void start(MetricProvider metricProvider, CfgT configuration);
 
     /**
      * Stop and cleanup work for current exporter must be implemented here.
@@ -60,7 +56,26 @@ public interface MetricExporter<CfgT extends ExporterView> {
     /**
      * Invokes, when exporter's configuration was updated.
      *
-     * @param newValue new configuration.
+     * <p>Be careful: this method will be invoked from the separate configuration events' thread pool.
+     * Appropriate thread-safe logic falls on the shoulders of implementations.
+     *
+     * @param newValue New configuration view.
      */
     void reconfigure(CfgT newValue);
+
+    /**
+     * {@link org.apache.ignite.internal.metrics.MetricManager} invokes this method,
+     * when new metric source was enabled.
+     *
+     * @param metricSet Named metric set.
+     */
+    void addMetricSet(MetricSet metricSet);
+
+    /**
+     * {@link org.apache.ignite.internal.metrics.MetricManager} invokes this method,
+     * when the metric source was disabled.
+     *
+     * @param metricSetName Name of metric set to remove.
+     */
+    void removeMetricSet(String metricSetName);
 }
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/PushMetricExporter.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/PushMetricExporter.java
index 7287c893bd..71704e05be 100644
--- a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/PushMetricExporter.java
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/PushMetricExporter.java
@@ -23,6 +23,7 @@ import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metrics.MetricProvider;
 import org.apache.ignite.internal.metrics.exporters.configuration.ExporterView;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -44,7 +45,9 @@ public abstract class PushMetricExporter<CfgT extends ExporterView> extends Basi
 
     /** {@inheritDoc} */
     @Override
-    public synchronized void start() {
+    public synchronized void start(MetricProvider metricProvider, CfgT conf) {
+        super.start(metricProvider, conf);
+
         scheduler =
                 Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("metrics-exporter-" + name(), log));
 
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java
index 68fe32e8c3..039aca7487 100644
--- a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.metrics.exporters.configuration;
 
+import static org.apache.ignite.internal.metrics.exporters.jmx.JmxExporter.JMX_EXPORTER_NAME;
+
 import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
 
 /**
  * Jmx exporter configuration.
  */
-// TODO: Should be clarified under IGNITE-17357.
-// TODO: Now it is needed to satisfy configuration framework, we can't have polymorphic config with 0 instances
-@PolymorphicConfigInstance("jmx")
+@PolymorphicConfigInstance(value = JMX_EXPORTER_NAME)
 public class JmxExporterConfigurationSchema extends ExporterConfigurationSchema {
 }
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/jmx/JmxExporter.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/jmx/JmxExporter.java
new file mode 100644
index 0000000000..35dfdaadc3
--- /dev/null
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/jmx/JmxExporter.java
@@ -0,0 +1,165 @@
+/*
+ * 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.ignite.internal.metrics.exporters.jmx;
+
+import static org.apache.ignite.internal.util.IgniteUtils.makeMbeanName;
+
+import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.List;
+import javax.management.JMException;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metrics.MetricProvider;
+import org.apache.ignite.internal.metrics.MetricSet;
+import org.apache.ignite.internal.metrics.exporters.BasicMetricExporter;
+import org.apache.ignite.internal.metrics.exporters.configuration.JmxExporterView;
+
+/**
+ * Exporter for Ignite metrics to JMX API.
+ * For each enabled {@link org.apache.ignite.internal.metrics.MetricSource} exporter provides
+ * a separate MBean with corresponding attribute per source's metric.
+ */
+public class JmxExporter extends BasicMetricExporter<JmxExporterView> {
+    /**
+     * Exporter name. Must be the same for configuration and exporter itself.
+     */
+    public static final String JMX_EXPORTER_NAME = "jmx";
+
+    /**
+     * Group attribute of {@link ObjectName} shared for all metric MBeans.
+     */
+    private static final String JMX_METRIC_GROUP = "metrics";
+
+    /**
+     * Logger.
+     */
+    private static IgniteLogger LOG = Loggers.forClass(JmxExporter.class);
+
+    /**
+     * Current registered MBeans.
+     */
+    private final List<ObjectName> mbeans = new ArrayList<>();
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public synchronized void start(MetricProvider metricsProvider, JmxExporterView configuration) {
+        super.start(metricsProvider, configuration);
+
+        for (MetricSet metricSet : metricsProvider.metrics().get1().values()) {
+            register(metricSet);
+        }
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public synchronized void stop() {
+        mbeans.forEach(this::unregBean);
+
+        mbeans.clear();
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String name() {
+        return JMX_EXPORTER_NAME;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * <p>Register new MBean for received metric set.
+     */
+    @Override
+    public synchronized void addMetricSet(MetricSet metricSet) {
+        register(metricSet);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * <p>Unregister MBean for removed metric set.
+     */
+    @Override
+    public synchronized void removeMetricSet(String metricSet) {
+        unregister(metricSet);
+    }
+
+    /**
+     * Register new MBean per metric set.
+     *
+     * @param metricSet Metric set.
+     */
+    private void register(MetricSet metricSet) {
+        try {
+            MetricSetMbean metricSetMbean = new MetricSetMbean(metricSet);
+
+            ObjectName mbean = ManagementFactory.getPlatformMBeanServer()
+                    .registerMBean(
+                            metricSetMbean,
+                            makeMbeanName(JMX_METRIC_GROUP, metricSet.name()))
+                    .getObjectName();
+
+            mbeans.add(mbean);
+        } catch (JMException e) {
+            LOG.error("MBean for metric set " + metricSet.name() + " can't be created.", e);
+        }
+    }
+
+    /**
+     * Unregister MBean for specific metric set.
+     *
+     * @param metricSetName Metric set name.
+     */
+    private void unregister(String metricSetName) {
+        try {
+            ObjectName mbeanName = makeMbeanName(JMX_METRIC_GROUP, metricSetName);
+
+            boolean rmv = mbeans.remove(mbeanName);
+
+            if (rmv) {
+                unregBean(mbeanName);
+            } else {
+                LOG.warn("Tried to unregister the MBean for non-registered metric set " + metricSetName);
+            }
+        } catch (MalformedObjectNameException e) {
+            LOG.error("MBean for metric set " + metricSetName + " can't be unregistered.", e);
+        }
+    }
+
+    /**
+     * Unregister MBean by its name.
+     *
+     * @param bean MBean name to unregister.
+     */
+    private void unregBean(ObjectName bean) {
+        try {
+            ManagementFactory.getPlatformMBeanServer().unregisterMBean(bean);
+        } catch (JMException e) {
+            LOG.error("Failed to unregister MBean: " + bean, e);
+        }
+    }
+}
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/jmx/MetricSetMbean.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/jmx/MetricSetMbean.java
new file mode 100644
index 0000000000..86da7a92ac
--- /dev/null
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/jmx/MetricSetMbean.java
@@ -0,0 +1,181 @@
+/*
+ * 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.ignite.internal.metrics.exporters.jmx;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import javax.management.Attribute;
+import javax.management.AttributeList;
+import javax.management.AttributeNotFoundException;
+import javax.management.DynamicMBean;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.ReflectionException;
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.DoubleMetric;
+import org.apache.ignite.internal.metrics.IntMetric;
+import org.apache.ignite.internal.metrics.LongMetric;
+import org.apache.ignite.internal.metrics.Metric;
+import org.apache.ignite.internal.metrics.MetricManager;
+import org.apache.ignite.internal.metrics.MetricSet;
+
+/**
+ * MBean implementation, which produce JMX API representation for {@link MetricSet}.
+ * Every {@link Metric} of metric set will be represented by MBean's attribute with the same name.
+ */
+public class MetricSetMbean implements DynamicMBean {
+    /**
+     * Metric set.
+     */
+    private MetricSet metricSet;
+
+    /**
+     * Constructs new MBean.
+     *
+     * @param metricSet Metric set.
+     */
+    public MetricSetMbean(MetricSet metricSet) {
+        this.metricSet = metricSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Object getAttribute(String attribute) throws AttributeNotFoundException {
+        if (attribute.equals("MBeanInfo")) {
+            return getMBeanInfo();
+        }
+
+        Metric metric = metricSet.get(attribute);
+
+        if (metric instanceof DoubleMetric) {
+            return ((DoubleMetric) metric).value();
+        } else if (metric instanceof IntMetric) {
+            return ((IntMetric) metric).value();
+        } else if (metric instanceof LongMetric) {
+            return ((LongMetric) metric).value();
+        } else if (metric instanceof DistributionMetric) {
+            return ((DistributionMetric) metric).value();
+        }
+
+        throw new AttributeNotFoundException("Unknown metric class " + metric.getClass());
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public AttributeList getAttributes(String[] attributes) {
+        AttributeList list = new AttributeList();
+
+        try {
+            for (String attribute : attributes) {
+                Object val = getAttribute(attribute);
+
+                list.add(val);
+            }
+        } catch (AttributeNotFoundException e) {
+            throw new IllegalArgumentException(e);
+        }
+
+        return list;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Object invoke(String actionName, Object[] params, String[] signature) throws MBeanException, ReflectionException {
+        if ("getAttribute".equals(actionName)) {
+            try {
+                return getAttribute((String) params[0]);
+            } catch (AttributeNotFoundException e) {
+                throw new MBeanException(e);
+            }
+        }
+
+        throw new UnsupportedOperationException("invoke is not supported.");
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public MBeanInfo getMBeanInfo() {
+        Iterator<Metric> iter = metricSet.iterator();
+
+        List<MBeanAttributeInfo> attrs = new ArrayList<>();
+
+        iter.forEachRemaining(metric -> {
+            attrs.add(new MBeanAttributeInfo(
+                    metric.name(),
+                    metricClass(metric),
+                    metric.description() != null ? metric.description() : metric.name(),
+                    true,
+                    false,
+                    false));
+        });
+
+        return new MBeanInfo(
+                MetricManager.class.getName(),
+                metricSet.name(),
+                attrs.toArray(new MBeanAttributeInfo[attrs.size()]),
+                null,
+                null,
+                null);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void setAttribute(Attribute attribute) {
+        throw new UnsupportedOperationException("setAttribute is not supported.");
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public AttributeList setAttributes(AttributeList attributes) {
+        throw new UnsupportedOperationException("setAttributes is not supported.");
+    }
+
+    /**
+     * Extract class of metric value.
+     *
+     * @param metric Metric.
+     * @return Class of metric value.
+     */
+    private String metricClass(Metric metric) {
+        if (metric instanceof DoubleMetric) {
+            return Double.class.getName();
+        } else if (metric instanceof IntMetric) {
+            return Integer.class.getName();
+        } else if (metric instanceof LongMetric) {
+            return Long.class.getName();
+        } else if (metric instanceof DistributionMetric) {
+            return long[].class.getName();
+        }
+
+        throw new IllegalArgumentException("Unknown metric class " + metric.getClass());
+    }
+}
diff --git a/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/JmxExporterTest.java b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/JmxExporterTest.java
new file mode 100644
index 0000000000..fb5e5de1dc
--- /dev/null
+++ b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/JmxExporterTest.java
@@ -0,0 +1,211 @@
+/*
+ * 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.ignite.internal.metrics;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.lang.management.ManagementFactory;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import javax.management.AttributeNotFoundException;
+import javax.management.DynamicMBean;
+import javax.management.InstanceNotFoundException;
+import javax.management.IntrospectionException;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.MBeanRegistrationException;
+import javax.management.MBeanServer;
+import javax.management.MBeanServerInvocationHandler;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.metrics.configuration.MetricConfiguration;
+import org.apache.ignite.internal.metrics.exporters.configuration.JmxExporterView;
+import org.apache.ignite.internal.metrics.exporters.jmx.JmxExporter;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Tests for {@link JmxExporter}.
+ */
+@ExtendWith({ConfigurationExtension.class})
+public class JmxExporterTest {
+    @InjectConfiguration(value = "mock.exporters = {jmx = {exporterName = jmx}}")
+    private MetricConfiguration metricConfiguration;
+
+    private JmxExporterView jmxExporterConf;
+
+    private static final MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer();
+
+    private static final String SRC_NAME = "testSource";
+
+    private static final String MTRC_NAME = "testMetric";
+
+    /**
+     * Metric set with all available metric types.
+     */
+    private static final MetricSet metricSet =
+            new MetricSet(
+                    SRC_NAME,
+                    Map.of(
+                            "intGauge", new IntGauge("intGauge", "", () -> 1),
+                            "longGauge", new LongGauge("longGauge", "", () -> 1L),
+                            "doubleGauge", new DoubleGauge("doubleGauge", "", () -> 1d),
+                            "atomicInt", new AtomicIntMetric("atomicInt", ""),
+                            "atomicLong", new AtomicLongMetric("atomicLong", ""),
+                            "atomicDouble", new AtomicDoubleMetric("atomicDouble", ""),
+                            "longAdder", new LongAdderMetric("longAdder", ""),
+                            "doubleAdder", new DoubleAdderMetric("doubleAdder", ""),
+                            "distributionMetric", new DistributionMetric("distributionMetric", "", new long[] {0, 1}),
+                            "hitRate", new HitRateMetric("hitRate", "", Long.MAX_VALUE)
+                    )
+            );
+
+    private ObjectName mbeanName;
+
+    private MetricProvider metricsProvider;
+
+    private JmxExporter jmxExporter;
+
+    @BeforeEach
+    void setUp() throws MalformedObjectNameException {
+        jmxExporterConf = (JmxExporterView) metricConfiguration.exporters().get("jmx").value();
+
+        mbeanName = IgniteUtils.makeMbeanName("metrics", SRC_NAME);
+
+        jmxExporter = new JmxExporter();
+
+        metricsProvider = mock(MetricProvider.class);
+    }
+
+    @AfterEach
+    void tearDown() throws MBeanRegistrationException {
+        try {
+            mbeanSrv.unregisterMBean(mbeanName);
+        } catch (InstanceNotFoundException e) {
+            // No op
+        }
+    }
+
+    @Test
+    public void testStart()
+            throws ReflectionException, AttributeNotFoundException, MBeanException {
+        Map<String, MetricSet> metrics = Map.of(metricSet.name(), metricSet);
+
+        when(metricsProvider.metrics()).thenReturn(new IgniteBiTuple<>(metrics, 1L));
+
+        jmxExporter.start(metricsProvider, jmxExporterConf);
+
+        assertThatMbeanAttributeAndMetricValuesAreTheSame();
+    }
+
+    @Test
+    public void testAddMetric()
+            throws ReflectionException, AttributeNotFoundException, MBeanException {
+        when(metricsProvider.metrics()).thenReturn(new IgniteBiTuple<>(new HashMap<>(), 1L));
+
+        jmxExporter.start(metricsProvider, jmxExporterConf);
+
+        assertThrows(
+                InstanceNotFoundException.class,
+                this::getMbeanInfo,
+                "Expected that mbean won't find, but it was");
+
+        jmxExporter.addMetricSet(metricSet);
+
+        assertThatMbeanAttributeAndMetricValuesAreTheSame();
+    }
+
+    @Test
+    public void testRemoveMetric()
+            throws ReflectionException, AttributeNotFoundException, MBeanException {
+        Map<String, MetricSet> metrics = Map.of(metricSet.name(), metricSet);
+
+        when(metricsProvider.metrics()).thenReturn(new IgniteBiTuple<>(metrics, 1L));
+
+        jmxExporter.start(metricsProvider, jmxExporterConf);
+
+        assertThatMbeanAttributeAndMetricValuesAreTheSame();
+
+        jmxExporter.removeMetricSet("testSource");
+
+        assertThrows(InstanceNotFoundException.class, this::getMbeanInfo,
+                "Expected that mbean won't find, but it was");
+    }
+
+    @Test
+    public void testMetricUpdate() throws ReflectionException, AttributeNotFoundException, MBeanException {
+        var intMetric = new AtomicIntMetric(MTRC_NAME, "");
+
+        MetricSet metricSet = new MetricSet(SRC_NAME, Map.of(MTRC_NAME, intMetric));
+
+        when(metricsProvider.metrics()).thenReturn(new IgniteBiTuple<>(Map.of(metricSet.name(), metricSet), 1L));
+
+        jmxExporter.start(metricsProvider, jmxExporterConf);
+
+        assertEquals(0, mbean().getAttribute(MTRC_NAME));
+
+        intMetric.add(1);
+
+        // test, that MBean has no stale metric value.
+        assertEquals(1, mbean().getAttribute(MTRC_NAME));
+    }
+
+    /**
+     * Check, that all MBean attributes has the same values as original metric values.
+     */
+    private void assertThatMbeanAttributeAndMetricValuesAreTheSame()
+            throws ReflectionException, AttributeNotFoundException, MBeanException {
+        for (Iterator<Metric> it = metricSet.iterator(); it.hasNext(); ) {
+            Metric metric = it.next();
+
+            Object beanAttribute = mbean().getAttribute(metric.name());
+
+            String errorMsg = "Wrong MBean attribute value for the metric with name " + metric.name();
+
+            if (metric instanceof IntMetric) {
+                assertEquals(((IntMetric) metric).value(), beanAttribute, errorMsg);
+            } else if (metric instanceof LongMetric) {
+                assertEquals(((LongMetric) metric).value(), beanAttribute, errorMsg);
+            } else if (metric instanceof DoubleMetric) {
+                assertEquals(((DoubleMetric) metric).value(), beanAttribute, errorMsg);
+            } else if (metric instanceof DistributionMetric) {
+                assertArrayEquals(((DistributionMetric) metric).value(), (long[]) beanAttribute, errorMsg);
+            }
+        }
+    }
+
+    private DynamicMBean mbean() {
+        return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, DynamicMBean.class, false);
+    }
+
+    private MBeanInfo getMbeanInfo() throws ReflectionException, InstanceNotFoundException, IntrospectionException {
+        return ManagementFactory.getPlatformMBeanServer().getMBeanInfo(mbeanName);
+    }
+}
diff --git a/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java
index d40b100911..b95795cf22 100644
--- a/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java
+++ b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.metrics.exporters;
 
 import org.apache.ignite.internal.metrics.MetricProvider;
+import org.apache.ignite.internal.metrics.MetricSet;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -30,14 +31,11 @@ public class TestExporter extends BasicMetricExporter<TestExporterView> {
     private volatile int port;
 
     @Override
-    public void init(MetricProvider metricsProvider, TestExporterView configuration) {
-        super.init(metricsProvider, configuration);
+    public void start(MetricProvider metricsProvider, TestExporterView configuration) {
+        super.start(metricsProvider, configuration);
 
         port = configuration.port();
-    }
 
-    @Override
-    public void start() {
         started = true;
     }
 
@@ -52,10 +50,20 @@ public class TestExporter extends BasicMetricExporter<TestExporterView> {
     }
 
     @Override
-    public void reconfigure(@Nullable TestExporterView newValue) {
+    public synchronized void reconfigure(@Nullable TestExporterView newValue) {
         super.reconfigure(newValue);
 
-        port = configuration.port();
+        port = configuration().port();
+    }
+
+    @Override
+    public void addMetricSet(MetricSet metricSet) {
+
+    }
+
+    @Override
+    public void removeMetricSet(String metricSetName) {
+
     }
 
     public boolean isStarted() {