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

[ignite-3] branch main updated: IGNITE-17358 Metric exporters' configurations. Fixes #1098

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

sk0x50 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 abe1ada882 IGNITE-17358 Metric exporters' configurations. Fixes #1098
abe1ada882 is described below

commit abe1ada8823d598cc81794f8c1fe73eaaad33d37
Author: Kirill Gusakov <kg...@gmail.com>
AuthorDate: Mon Oct 3 11:49:47 2022 +0300

    IGNITE-17358 Metric exporters' configurations. Fixes #1098
    
    Signed-off-by: Slava Koptilin <sl...@gmail.com>
---
 modules/metrics/build.gradle                       |   3 +
 modules/metrics/pom.xml                            |  48 ++++++++
 ...Test.java => ItMetricExportersLoadingTest.java} |  21 +++-
 .../metrics/exporters/TestPullMetricExporter.java  |   9 +-
 ...estPullMetricsExporterConfigurationSchema.java} |  20 ++--
 .../metrics/exporters/TestPushMetricExporter.java  |  24 +++-
 ...estPushMetricsExporterConfigurationSchema.java} |  20 ++--
 .../ignite/internal/metrics/MetricManager.java     | 121 ++++++++++++++++++---
 .../configuration/MetricConfigurationModule.java   |  59 ++++++++++
 .../configuration/MetricConfigurationSchema.java}  |  22 ++--
 .../metrics/exporters/BasicMetricExporter.java     |  15 ++-
 .../internal/metrics/exporters/MetricExporter.java |  16 ++-
 .../metrics/exporters/PushMetricExporter.java      |  42 +++----
 .../ExporterConfigurationSchema.java}              |  25 +++--
 .../JmxExporterConfigurationSchema.java}           |  18 +--
 ...nite.internal.configuration.ConfigurationModule |  17 +++
 .../internal/metrics/MetricConfigurationTest.java  | 111 +++++++++++++++++++
 .../internal/metrics/exporters/TestExporter.java}  |  56 ++++++----
 .../TestExporterConfigurationSchema.java}          |  20 ++--
 .../org/apache/ignite/internal/app/IgniteImpl.java |   5 +-
 20 files changed, 539 insertions(+), 133 deletions(-)

diff --git a/modules/metrics/build.gradle b/modules/metrics/build.gradle
index d1f08e7b80..89507df1f2 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/build.gradle
@@ -21,9 +21,12 @@ apply from: "$rootDir/buildscripts/java-integration-test.gradle"
 
 dependencies {
     implementation project(':ignite-core')
+    implementation project(':ignite-configuration')
+    implementation project(':ignite-configuration-api')
     implementation libs.jetbrains.annotations
 
     testImplementation libs.hamcrest.core
+    testImplementation libs.mockito.core
 }
 
 description = 'ignite-metrics'
diff --git a/modules/metrics/pom.xml b/modules/metrics/pom.xml
index 720e99cefd..28e5c27c2e 100644
--- a/modules/metrics/pom.xml
+++ b/modules/metrics/pom.xml
@@ -38,6 +38,16 @@
             <artifactId>ignite-core</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-configuration</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-configuration-api</artifactId>
+        </dependency>
+
         <!-- 3rd party dependencies -->
         <dependency>
             <groupId>org.jetbrains</groupId>
@@ -51,6 +61,12 @@
             <scope>test</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
         <dependency>
             <groupId>org.junit.jupiter</groupId>
             <artifactId>junit-jupiter-api</artifactId>
@@ -62,5 +78,37 @@
             <artifactId>junit-jupiter-engine</artifactId>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-configuration</artifactId>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.ignite</groupId>
+                        <artifactId>ignite-configuration-annotation-processor</artifactId>
+                        <version>${project.version}</version>
+                    </dependency>
+                </dependencies>
+                <configuration>
+                    <annotationProcessorPaths>
+                        <path>
+                            <groupId>org.apache.ignite</groupId>
+                            <artifactId>ignite-configuration-annotation-processor</artifactId>
+                            <version>${project.version}</version>
+                        </path>
+                    </annotationProcessorPaths>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 </project>
diff --git a/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/MetricExportersLoadingTest.java b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/ItMetricExportersLoadingTest.java
similarity index 74%
rename from modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/MetricExportersLoadingTest.java
rename to modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/ItMetricExportersLoadingTest.java
index 800c0c9f6a..500582fa28 100644
--- a/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/MetricExportersLoadingTest.java
+++ b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/ItMetricExportersLoadingTest.java
@@ -23,17 +23,36 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 import java.io.ByteArrayOutputStream;
 import java.io.OutputStream;
 import java.util.concurrent.locks.LockSupport;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.metrics.MetricManager;
+import org.apache.ignite.internal.metrics.configuration.MetricConfiguration;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * Integration test for metrics' exporters loading.
  */
-public class MetricExportersLoadingTest {
+@ExtendWith({ConfigurationExtension.class})
+public class ItMetricExportersLoadingTest {
+    @InjectConfiguration(
+            value = "mock.exporters = {"
+                    + "testPull = {exporterName = testPull},"
+                    + "testPush = {exporterName = testPush, period = 100},"
+                    + "}",
+            polymorphicExtensions = {
+                    TestPushMetricsExporterConfigurationSchema.class,
+                    TestPullMetricsExporterConfigurationSchema.class
+            }
+    )
+    private MetricConfiguration metricConfiguration;
+
     @Test
     public void test() throws Exception {
         MetricManager metricManager = new MetricManager();
 
+        metricManager.configure(metricConfiguration);
+
         TestMetricsSource src = new TestMetricsSource("TestMetricsSource");
 
         metricManager.registerSource(src);
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 2816388a60..2d2da80e18 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
@@ -29,7 +29,9 @@ import org.apache.ignite.internal.metrics.MetricSet;
  * Simple pull exporter, which simulate the pull principe throw primitive wait/notify API
  * instead of the complex TCP/IP etc. endpoints.
  */
-public class TestPullMetricExporter extends BasicMetricExporter {
+public class TestPullMetricExporter extends BasicMetricExporter<TestPullMetricsExporterView> {
+    public static final String EXPORTER_NAME = "testPull";
+
     private static OutputStream outputStream;
 
     private static final Object obj = new Object();
@@ -86,6 +88,11 @@ public class TestPullMetricExporter extends BasicMetricExporter {
         executorService.shutdown();
     }
 
+    @Override
+    public String name() {
+        return EXPORTER_NAME;
+    }
+
     private void waitForRequest() {
         synchronized (obj) {
             try {
diff --git a/modules/metrics/build.gradle b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricsExporterConfigurationSchema.java
similarity index 59%
copy from modules/metrics/build.gradle
copy to modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricsExporterConfigurationSchema.java
index d1f08e7b80..eacc03b1a4 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPullMetricsExporterConfigurationSchema.java
@@ -15,15 +15,17 @@
  * limitations under the License.
  */
 
-apply from: "$rootDir/buildscripts/java-core.gradle"
-apply from: "$rootDir/buildscripts/java-junit5.gradle"
-apply from: "$rootDir/buildscripts/java-integration-test.gradle"
+package org.apache.ignite.internal.metrics.exporters;
 
-dependencies {
-    implementation project(':ignite-core')
-    implementation libs.jetbrains.annotations
+import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterConfigurationSchema;
 
-    testImplementation libs.hamcrest.core
+/**
+ * Configuration for test pull exporter.
+ */
+@PolymorphicConfigInstance(TestPullMetricExporter.EXPORTER_NAME)
+public class TestPullMetricsExporterConfigurationSchema extends ExporterConfigurationSchema {
+    @Value(hasDefault = true)
+    public int port = 11111;
 }
-
-description = 'ignite-metrics'
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 57f054b9e3..668ff987e5 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
@@ -21,22 +21,35 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import org.apache.ignite.internal.metrics.Metric;
+import org.apache.ignite.internal.metrics.MetricProvider;
 import org.apache.ignite.internal.metrics.MetricSet;
 
 /**
  * Test push metrics exporter.
  */
-public class TestPushMetricExporter extends PushMetricExporter {
+public class TestPushMetricExporter extends PushMetricExporter<TestPushMetricsExporterView> {
+    public static final String EXPORTER_NAME = "testPush";
+
     private static OutputStream outputStream;
 
-    public TestPushMetricExporter() {
-        setPeriod(100);
+    private long period;
+
+    @Override
+    public void init(MetricProvider metricsProvider, TestPushMetricsExporterView configuration) {
+        super.init(metricsProvider, configuration);
+
+        period = configuration.period();
     }
 
     public static void setOutputStream(OutputStream outputStream) {
         TestPushMetricExporter.outputStream = outputStream;
     }
 
+    @Override
+    protected long period() {
+        return period;
+    }
+
     @Override
     public void report() {
         var report = new StringBuilder();
@@ -66,4 +79,9 @@ public class TestPushMetricExporter extends PushMetricExporter {
             throw new RuntimeException(e);
         }
     }
+
+    @Override
+    public String name() {
+        return EXPORTER_NAME;
+    }
 }
diff --git a/modules/metrics/build.gradle b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricsExporterConfigurationSchema.java
similarity index 59%
copy from modules/metrics/build.gradle
copy to modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricsExporterConfigurationSchema.java
index d1f08e7b80..3b07fc5545 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/src/integrationTest/java/org/apache/ignite/internal/metrics/exporters/TestPushMetricsExporterConfigurationSchema.java
@@ -15,15 +15,17 @@
  * limitations under the License.
  */
 
-apply from: "$rootDir/buildscripts/java-core.gradle"
-apply from: "$rootDir/buildscripts/java-junit5.gradle"
-apply from: "$rootDir/buildscripts/java-integration-test.gradle"
+package org.apache.ignite.internal.metrics.exporters;
 
-dependencies {
-    implementation project(':ignite-core')
-    implementation libs.jetbrains.annotations
+import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterConfigurationSchema;
 
-    testImplementation libs.hamcrest.core
+/**
+ * Configuration for test push exporter.
+ */
+@PolymorphicConfigInstance(TestPushMetricExporter.EXPORTER_NAME)
+public class TestPushMetricsExporterConfigurationSchema extends ExporterConfigurationSchema {
+    @Value(hasDefault = true)
+    public int period = 300;
 }
-
-description = 'ignite-metrics'
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 b25315a638..b4f2ca0e0e 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
@@ -17,54 +17,93 @@
 
 package org.apache.ignite.internal.metrics;
 
-import static java.util.stream.Collectors.toUnmodifiableList;
-
-import java.util.List;
 import java.util.Map;
 import java.util.ServiceLoader;
 import java.util.ServiceLoader.Provider;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
+import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.metrics.configuration.MetricConfiguration;
+import org.apache.ignite.internal.metrics.configuration.MetricView;
 import org.apache.ignite.internal.metrics.exporters.MetricExporter;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterView;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.VisibleForTesting;
 
 
 /**
  * Metric manager.
  */
 public class MetricManager implements IgniteComponent {
-    /**
-     * Metric registry.
-     */
+    /** Logger. */
+    private static final IgniteLogger LOG = Loggers.forClass(MetricManager.class);
+
+    /** Metric registry. */
     private final MetricRegistry registry;
 
+    private final MetricProvider metricsProvider;
+
+    private final Map<String, MetricExporter> enabledMetricExporters = new ConcurrentHashMap<>();
+
     /** Metrics' exporters. */
-    private List<MetricExporter> metricExporters;
+    private Map<String, MetricExporter> availableExporters;
+
+    private MetricConfiguration metricConfiguration;
 
     /**
      * Constructor.
      */
     public MetricManager() {
-        this.registry = new MetricRegistry();
+        registry = new MetricRegistry();
+        metricsProvider = new MetricProvider(registry);
+    }
+
+    /**
+     * Method to configure {@link MetricManager} with distributed configuration.
+     *
+     * @param metricConfiguration Distributed metric configuration.
+     */
+    // TODO: IGNITE-17718 when we design the system to configure metrics itself
+    // TODO: this method should be revisited, but now it is supposed to use only to set distributed configuration for exporters.
+    public void configure(MetricConfiguration metricConfiguration) {
+        assert this.metricConfiguration == null : "Metric manager must be configured only once, on the start of the node";
+
+        this.metricConfiguration = metricConfiguration;
     }
 
     /** {@inheritDoc} */
     @Override public void start() {
-        // TODO: IGNITE-17358 not all exporters should be started, it must be defined by configuration
-        metricExporters = loadExporters();
+        start(loadExporters());
+    }
 
-        MetricProvider metricsProvider = new MetricProvider(registry);
+    /**
+     * Start component.
+     *
+     * @param availableExporters Map of (name, exporter) with available exporters.
+     */
+    @VisibleForTesting
+    public void start(Map<String, MetricExporter> availableExporters) {
+        this.availableExporters = availableExporters;
 
-        for (MetricExporter metricExporter : metricExporters) {
-            metricExporter.init(metricsProvider);
+        MetricView conf = metricConfiguration.value();
 
-            metricExporter.start();
+        for (String exporterName : conf.exporters().namedListKeys()) {
+            checkAndStartExporter(exporterName, metricConfiguration.exporters().get(exporterName).value());
         }
+
+        metricConfiguration.exporters().listenElements(new ExporterConfigurationListener());
     }
 
     /** {@inheritDoc} */
     @Override public void stop() throws Exception {
-        for (MetricExporter metricExporter : metricExporters) {
+        for (MetricExporter metricExporter : enabledMetricExporters.values()) {
             metricExporter.stop();
         }
     }
@@ -121,14 +160,14 @@ public class MetricManager implements IgniteComponent {
      *
      * @return list of loaded exporters.
      */
-    private List<MetricExporter> loadExporters() {
+    public static Map<String, MetricExporter> loadExporters() {
         var clsLdr = Thread.currentThread().getContextClassLoader();
 
         return ServiceLoader
                 .load(MetricExporter.class, clsLdr)
                 .stream()
                 .map(Provider::get)
-                .collect(toUnmodifiableList());
+                .collect(Collectors.toMap(e -> e.name(), Function.identity()));
     }
 
     /**
@@ -159,4 +198,52 @@ public class MetricManager implements IgniteComponent {
     public IgniteBiTuple<Map<String, MetricSet>, Long> metricSnapshot() {
         return registry.metricSnapshot();
     }
+
+    private <T extends ExporterView> void checkAndStartExporter(
+            String exporterName,
+            T exporterConfiguration) {
+        MetricExporter<T> exporter = availableExporters.get(exporterName);
+
+        if (exporter != null) {
+            exporter.init(metricsProvider, exporterConfiguration);
+
+            exporter.start();
+
+            enabledMetricExporters.put(exporter.name(), exporter);
+        } else {
+            LOG.warn("Received configuration for unknown metric exporter with the name '" + exporterName + "'");
+        }
+
+    }
+
+    private class ExporterConfigurationListener implements ConfigurationNamedListListener<ExporterView> {
+        @Override
+        public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<ExporterView> ctx) {
+            checkAndStartExporter(ctx.newValue().exporterName(), ctx.newValue());
+
+            return CompletableFuture.completedFuture(null);
+        }
+
+        @Override
+        public CompletableFuture<?> onDelete(ConfigurationNotificationEvent<ExporterView> ctx) {
+            var removed = enabledMetricExporters.remove(ctx.oldValue().exporterName());
+
+            if (removed != null) {
+                removed.stop();
+            }
+
+            return CompletableFuture.completedFuture(null);
+        }
+
+        @Override
+        public CompletableFuture<?> onUpdate(ConfigurationNotificationEvent<ExporterView> ctx) {
+            MetricExporter exporter = enabledMetricExporters.get(ctx.newValue().exporterName());
+
+            if (exporter != null) {
+                exporter.reconfigure(ctx.newValue());
+            }
+
+            return CompletableFuture.completedFuture(null);
+        }
+    }
 }
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/configuration/MetricConfigurationModule.java b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/configuration/MetricConfigurationModule.java
new file mode 100644
index 0000000000..c8f0920c5b
--- /dev/null
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/configuration/MetricConfigurationModule.java
@@ -0,0 +1,59 @@
+/*
+ * 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.configuration;
+
+import java.lang.annotation.Annotation;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.annotation.ConfigurationType;
+import org.apache.ignite.configuration.validation.Validator;
+import org.apache.ignite.internal.configuration.ConfigurationModule;
+import org.apache.ignite.internal.metrics.exporters.configuration.JmxExporterConfigurationSchema;
+
+/**
+ * Configuration module for metrics' configs.
+ */
+public class MetricConfigurationModule implements ConfigurationModule {
+    /** {@inheritDoc} */
+    @Override
+    public ConfigurationType type() {
+        return ConfigurationType.DISTRIBUTED;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Collection<RootKey<?, ?>> rootKeys() {
+        return List.of(MetricConfiguration.KEY);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Collection<Class<?>> polymorphicSchemaExtensions() {
+        return List.of(JmxExporterConfigurationSchema.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Map<Class<? extends Annotation>, Set<Validator<? extends Annotation, ?>>> validators() {
+        return Collections.emptyMap();
+    }
+}
diff --git a/modules/metrics/build.gradle b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/configuration/MetricConfigurationSchema.java
similarity index 55%
copy from modules/metrics/build.gradle
copy to modules/metrics/src/main/java/org/apache/ignite/internal/metrics/configuration/MetricConfigurationSchema.java
index d1f08e7b80..a40d24d244 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/configuration/MetricConfigurationSchema.java
@@ -15,15 +15,19 @@
  * limitations under the License.
  */
 
-apply from: "$rootDir/buildscripts/java-core.gradle"
-apply from: "$rootDir/buildscripts/java-junit5.gradle"
-apply from: "$rootDir/buildscripts/java-integration-test.gradle"
+package org.apache.ignite.internal.metrics.configuration;
 
-dependencies {
-    implementation project(':ignite-core')
-    implementation libs.jetbrains.annotations
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.ConfigurationType;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterConfigurationSchema;
 
-    testImplementation libs.hamcrest.core
+/**
+ * Metrics' configuration root.
+ */
+@ConfigurationRoot(rootName = "metrics", type = ConfigurationType.DISTRIBUTED)
+public class MetricConfigurationSchema {
+    /** Map of exporters' configurations. */
+    @NamedConfigValue
+    public ExporterConfigurationSchema exporters;
 }
-
-description = 'ignite-metrics'
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 800eb32741..85ef05cd22 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
@@ -20,25 +20,30 @@ package org.apache.ignite.internal.metrics.exporters;
 import java.util.Map;
 import org.apache.ignite.internal.metrics.MetricProvider;
 import org.apache.ignite.internal.metrics.MetricSet;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterView;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Base class for new metrics exporters implementations.
  */
-public abstract class BasicMetricExporter implements MetricExporter {
+public abstract class BasicMetricExporter<CfgT extends ExporterView> implements MetricExporter<CfgT> {
     /** Metrics provider. */
     private MetricProvider metricsProvider;
 
+    protected CfgT configuration;
+
     /** {@inheritDoc} */
     @Override
-    public final void init(MetricProvider metricProvider) {
-        this.metricsProvider = metricProvider;
+    public void init(MetricProvider metricsProvider, CfgT configuration) {
+        this.metricsProvider = metricsProvider;
+        this.configuration = configuration;
     }
 
     /** {@inheritDoc} */
     @Override
-    public final String name() {
-        return getClass().getCanonicalName();
+    public void reconfigure(@Nullable CfgT newValue) {
+        configuration = newValue;
     }
 
     /**
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 fd2b7a271f..55759fd4d2 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,8 @@
 package org.apache.ignite.internal.metrics.exporters;
 
 import org.apache.ignite.internal.metrics.MetricProvider;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterConfiguration;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterView;
 
 /**
  * Interface for metric exporters to external recipients.
@@ -29,13 +31,14 @@ import org.apache.ignite.internal.metrics.MetricProvider;
  * <p>Pull exporters is the endpoint by itself (HTTP, JMX and etc.), which response with the metric data for request.
  * Pull exporters should extend {@link BasicMetricExporter}.
  */
-public interface MetricExporter {
+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);
+    void init(MetricProvider metricProvider, CfgT configuration);
 
     /**
      * Start metrics exporter. Here all needed listeners, schedulers etc. should be started.
@@ -48,9 +51,16 @@ public interface MetricExporter {
     void stop();
 
     /**
-     * Returns the name of exporter. Name must be unique.
+     * Returns the name of exporter. Name must be unique and be the same for exporter and its {@link ExporterConfiguration}.
      *
      * @return Name of the exporter.
      */
     String name();
+
+    /**
+     * Invokes, when exporter's configuration was updated.
+     *
+     * @param newValue new configuration.
+     */
+    void reconfigure(CfgT newValue);
 }
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 7c900d774c..7287c893bd 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.exporters.configuration.ExporterView;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
 
@@ -31,16 +32,10 @@ import org.apache.ignite.internal.util.IgniteUtils;
  * Every {@code period} of time {@link PushMetricExporter#report()} will be called
  * to push metrics to the external system.
  */
-public abstract class PushMetricExporter extends BasicMetricExporter {
+public abstract class PushMetricExporter<CfgT extends ExporterView> extends BasicMetricExporter<CfgT> {
     /** Logger. */
     protected final IgniteLogger log = Loggers.forClass(getClass());
 
-    /** Default export period in milliseconds. */
-    public static final long DFLT_EXPORT_PERIOD = 60_000;
-
-    /** Export period. */
-    private long period = DFLT_EXPORT_PERIOD;
-
     /** Export task future. */
     private ScheduledFuture<?> fut;
 
@@ -49,9 +44,9 @@ public abstract class PushMetricExporter extends BasicMetricExporter {
 
     /** {@inheritDoc} */
     @Override
-    public void start() {
+    public synchronized void start() {
         scheduler =
-                Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("metrics-exporter", log));
+                Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("metrics-exporter-" + name(), log));
 
         fut = scheduler.scheduleWithFixedDelay(() -> {
             try {
@@ -62,27 +57,20 @@ public abstract class PushMetricExporter extends BasicMetricExporter {
 
                 throw th;
             }
-        }, period, period, TimeUnit.MILLISECONDS);
+        }, period(), period(), TimeUnit.MILLISECONDS);
 
     }
 
     /** {@inheritDoc} */
     @Override
-    public void stop() {
-        fut.cancel(false);
-
-        IgniteUtils.shutdownAndAwaitTermination(scheduler, 10, TimeUnit.SECONDS);
-    }
-
-    // TODO: after IGNITE-17358 this period maybe shouldn't be a part of protected API
-    // TODO: and should be configured throw configuration API
-    /**
-     * Sets period in milliseconds after {@link #report()} method should be called.
-     *
-     * @param period Period in milliseconds.
-     */
-    protected void setPeriod(long period) {
-        this.period = period;
+    public synchronized void stop() {
+        if (fut != null) {
+            fut.cancel(false);
+        }
+
+        if (scheduler != null) {
+            IgniteUtils.shutdownAndAwaitTermination(scheduler, 10, TimeUnit.SECONDS);
+        }
     }
 
     /**
@@ -90,9 +78,7 @@ public abstract class PushMetricExporter extends BasicMetricExporter {
      *
      * @return Period in milliseconds after {@link #report()} method should be called.
      */
-    public long getPeriod() {
-        return period;
-    }
+    protected abstract long period();
 
     /**
      * A heart of the push exporter.
diff --git a/modules/metrics/build.gradle b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/ExporterConfigurationSchema.java
similarity index 50%
copy from modules/metrics/build.gradle
copy to modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/ExporterConfigurationSchema.java
index d1f08e7b80..1a2f35bdbe 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/ExporterConfigurationSchema.java
@@ -15,15 +15,22 @@
  * limitations under the License.
  */
 
-apply from: "$rootDir/buildscripts/java-core.gradle"
-apply from: "$rootDir/buildscripts/java-junit5.gradle"
-apply from: "$rootDir/buildscripts/java-integration-test.gradle"
+package org.apache.ignite.internal.metrics.exporters.configuration;
 
-dependencies {
-    implementation project(':ignite-core')
-    implementation libs.jetbrains.annotations
+import org.apache.ignite.configuration.annotation.PolymorphicConfig;
+import org.apache.ignite.configuration.annotation.PolymorphicId;
+import org.apache.ignite.internal.metrics.exporters.MetricExporter;
 
-    testImplementation libs.hamcrest.core
+/**
+ * Parent for any exporter configuration.
+ */
+// TODO: IGNITE-17721 at the moment we need to set exporter name twice: here and as the key on named list in MetricConfigurationSchema,
+// because we can't use AbstractConfiguration instead of PolymorphicConfig inside of NamedConfigValue
+@PolymorphicConfig
+public class ExporterConfigurationSchema {
+    /**
+     * The unique name of appropriate {@link MetricExporter}. It must be the same as {@link MetricExporter#name()}.
+     */
+    @PolymorphicId
+    public String exporterName;
 }
-
-description = 'ignite-metrics'
diff --git a/modules/metrics/build.gradle b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java
similarity index 63%
copy from modules/metrics/build.gradle
copy to modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java
index d1f08e7b80..68fe32e8c3 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/configuration/JmxExporterConfigurationSchema.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-apply from: "$rootDir/buildscripts/java-core.gradle"
-apply from: "$rootDir/buildscripts/java-junit5.gradle"
-apply from: "$rootDir/buildscripts/java-integration-test.gradle"
+package org.apache.ignite.internal.metrics.exporters.configuration;
 
-dependencies {
-    implementation project(':ignite-core')
-    implementation libs.jetbrains.annotations
+import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
 
-    testImplementation libs.hamcrest.core
+/**
+ * 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")
+public class JmxExporterConfigurationSchema extends ExporterConfigurationSchema {
 }
-
-description = 'ignite-metrics'
diff --git a/modules/metrics/src/main/resources/META-INF/services/org.apache.ignite.internal.configuration.ConfigurationModule b/modules/metrics/src/main/resources/META-INF/services/org.apache.ignite.internal.configuration.ConfigurationModule
new file mode 100644
index 0000000000..a24e2960a5
--- /dev/null
+++ b/modules/metrics/src/main/resources/META-INF/services/org.apache.ignite.internal.configuration.ConfigurationModule
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+org.apache.ignite.internal.metrics.configuration.MetricConfigurationModule
diff --git a/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/MetricConfigurationTest.java b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/MetricConfigurationTest.java
new file mode 100644
index 0000000000..4b2fb531b9
--- /dev/null
+++ b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/MetricConfigurationTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+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.MetricExporter;
+import org.apache.ignite.internal.metrics.exporters.TestExporter;
+import org.apache.ignite.internal.metrics.exporters.TestExporterChange;
+import org.apache.ignite.internal.metrics.exporters.TestExporterConfigurationSchema;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Tests for exporters' configurations.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public class MetricConfigurationTest {
+    @InjectConfiguration(
+            polymorphicExtensions = {
+                    TestExporterConfigurationSchema.class
+            }
+    )
+    private MetricConfiguration metricConfiguration;
+
+    private MetricManager metricManager;
+
+    private TestExporter exporter;
+
+    @BeforeEach
+    public void setUp() {
+        metricManager = new MetricManager();
+
+        Map<String, MetricExporter> availableExporters = new HashMap<>();
+
+        exporter  = new TestExporter();
+
+        availableExporters.put("test", exporter);
+
+        metricManager.configure(metricConfiguration);
+
+        metricManager.start(availableExporters);
+    }
+
+    @Test
+    public void testExporterStartStop() {
+        assertFalse(exporter.isStarted());
+
+        metricConfiguration.exporters().change(ch -> {
+            ch.create("test", exporterChange -> {
+                exporterChange.convert("test");
+            });
+        }).join();
+
+        assertTrue(exporter.isStarted());
+
+        metricConfiguration.exporters().change(ch -> {
+            ch.delete("test");
+        }).join();
+
+        assertFalse(exporter.isStarted());
+    }
+
+    @Test
+    public void testExporterReconfiguration() {
+        metricConfiguration.exporters().change(ch -> {
+            ch.create("test", exporterChange -> {
+                exporterChange.convert("test");
+            });
+        }).join();
+
+        assertEquals(0, exporter.port());
+
+        metricConfiguration.exporters().change(ch -> {
+            ch.update("test", exporterChange -> {
+                ((TestExporterChange) exporterChange.convert("test")).changePort(32);
+            });
+        }).join();
+
+        assertEquals(32, exporter.port());
+
+        metricConfiguration.exporters().get("test").change(ch -> {
+            ((TestExporterChange) ch).changePort(33);
+        }).join();
+
+        assertEquals(33, exporter.port());
+    }
+}
diff --git a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java
similarity index 51%
copy from modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java
copy to modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java
index 800eb32741..d40b100911 100644
--- a/modules/metrics/src/main/java/org/apache/ignite/internal/metrics/exporters/BasicMetricExporter.java
+++ b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporter.java
@@ -17,36 +17,52 @@
 
 package org.apache.ignite.internal.metrics.exporters;
 
-import java.util.Map;
 import org.apache.ignite.internal.metrics.MetricProvider;
-import org.apache.ignite.internal.metrics.MetricSet;
-import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Base class for new metrics exporters implementations.
+ * Simple test exporter with 1 configuration parameter and "started" flag.
  */
-public abstract class BasicMetricExporter implements MetricExporter {
-    /** Metrics provider. */
-    private MetricProvider metricsProvider;
+public class TestExporter extends BasicMetricExporter<TestExporterView> {
+
+    private volatile boolean started = false;
+
+    private volatile int port;
+
+    @Override
+    public void init(MetricProvider metricsProvider, TestExporterView configuration) {
+        super.init(metricsProvider, configuration);
+
+        port = configuration.port();
+    }
 
-    /** {@inheritDoc} */
     @Override
-    public final void init(MetricProvider metricProvider) {
-        this.metricsProvider = metricProvider;
+    public void start() {
+        started = true;
     }
 
-    /** {@inheritDoc} */
     @Override
-    public final String name() {
-        return getClass().getCanonicalName();
+    public void stop() {
+        started = false;
+    }
+
+    @Override
+    public String name() {
+        return "test";
+    }
+
+    @Override
+    public void reconfigure(@Nullable TestExporterView newValue) {
+        super.reconfigure(newValue);
+
+        port = configuration.port();
+    }
+
+    public boolean isStarted() {
+        return started;
     }
 
-    /**
-     * Returns a map of (metricSetName -> metricSet) pairs with available metrics.
-     *
-     * @return map of metrics
-     */
-    protected final IgniteBiTuple<Map<String, MetricSet>, Long> metrics() {
-        return metricsProvider.metrics();
+    public int port() {
+        return port;
     }
 }
diff --git a/modules/metrics/build.gradle b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporterConfigurationSchema.java
similarity index 61%
copy from modules/metrics/build.gradle
copy to modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporterConfigurationSchema.java
index d1f08e7b80..d8b21904ae 100644
--- a/modules/metrics/build.gradle
+++ b/modules/metrics/src/test/java/org/apache/ignite/internal/metrics/exporters/TestExporterConfigurationSchema.java
@@ -15,15 +15,17 @@
  * limitations under the License.
  */
 
-apply from: "$rootDir/buildscripts/java-core.gradle"
-apply from: "$rootDir/buildscripts/java-junit5.gradle"
-apply from: "$rootDir/buildscripts/java-integration-test.gradle"
+package org.apache.ignite.internal.metrics.exporters;
 
-dependencies {
-    implementation project(':ignite-core')
-    implementation libs.jetbrains.annotations
+import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.metrics.exporters.configuration.ExporterConfigurationSchema;
 
-    testImplementation libs.hamcrest.core
+/**
+ * Configuration for test exporter.
+ */
+@PolymorphicConfigInstance("test")
+public class TestExporterConfigurationSchema extends ExporterConfigurationSchema {
+    @Value(hasDefault = true)
+    public int port = 0;
 }
-
-description = 'ignite-metrics'
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index d00fe65cff..4040f5a865 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -60,6 +60,7 @@ import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
 import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
 import org.apache.ignite.internal.metrics.MetricManager;
+import org.apache.ignite.internal.metrics.configuration.MetricConfiguration;
 import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.raft.storage.impl.VolatileLogStorageFactoryCreator;
 import org.apache.ignite.internal.recovery.ConfigurationCatchUpListener;
@@ -292,6 +293,8 @@ public class IgniteImpl implements Ignite {
                 modules.distributed().polymorphicSchemaExtensions()
         );
 
+        metricManager.configure(clusterCfgMgr.configurationRegistry().getConfiguration(MetricConfiguration.KEY));
+
         RestFactory presentationsFactory = new PresentationsFactory(nodeCfgMgr, clusterCfgMgr);
         RestFactory clusterManagementRestFactory = new ClusterManagementRestFactory(clusterSvc, cmgMgr);
         RestFactory nodeManagementRestFactory = new NodeManagementRestFactory(lifecycleManager, () -> name);
@@ -438,7 +441,6 @@ public class IgniteImpl implements Ignite {
 
             // Start the components that are required to join the cluster.
             lifecycleManager.startComponents(
-                    metricManager,
                     nettyBootstrapFactory,
                     clusterSvc,
                     restComponent,
@@ -458,6 +460,7 @@ public class IgniteImpl implements Ignite {
                             lifecycleManager.startComponents(
                                     metaStorageMgr,
                                     clusterCfgMgr,
+                                    metricManager,
                                     computeComponent,
                                     txManager,
                                     baselineMgr,