You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by is...@apache.org on 2021/07/30 02:28:51 UTC

[ignite] branch master updated: IGNITE-14815 Metrics for thin clients

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7f5b6e7  IGNITE-14815 Metrics for thin clients
7f5b6e7 is described below

commit 7f5b6e73717faa9a9ddc91336caad11d0b6c9542
Author: Igor Sapego <is...@apache.org>
AuthorDate: Fri Jul 30 05:27:20 2021 +0300

    IGNITE-14815 Metrics for thin clients
    
    This closes #9182
---
 docs/_docs/monitoring-metrics/new-metrics.adoc     |   9 +-
 .../ignite/common/NodeSslConnectionMetricTest.java |   8 +-
 .../processors/metric/GridMetricManager.java       |   3 +
 .../odbc/ClientListenerConnectionContext.java      |   5 +
 .../processors/odbc/ClientListenerMetrics.java     | 155 +++++++++++++++
 .../processors/odbc/ClientListenerNioListener.java |  20 +-
 .../processors/odbc/ClientListenerProcessor.java   |  11 +-
 .../odbc/jdbc/JdbcConnectionContext.java           |  10 +-
 .../odbc/odbc/OdbcConnectionContext.java           |  11 +-
 .../platform/client/ClientConnectionContext.java   |   6 +
 .../rest/protocols/tcp/GridTcpRestProtocol.java    |   2 +-
 .../processors/odbc/ClientListenerMetricsTest.java | 221 +++++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java    |   3 +
 .../cpp/thin-client-test/include/test_utils.h      |  30 +++
 .../thin-client-test/src/ignite_client_test.cpp    |  24 +++
 15 files changed, 497 insertions(+), 21 deletions(-)

diff --git a/docs/_docs/monitoring-metrics/new-metrics.adoc b/docs/_docs/monitoring-metrics/new-metrics.adoc
index 36df674..1168709 100644
--- a/docs/_docs/monitoring-metrics/new-metrics.adoc
+++ b/docs/_docs/monitoring-metrics/new-metrics.adoc
@@ -294,7 +294,7 @@ Register name: `io.communication`
 
 == Ignite Thin Client Connector
 
-Register name: `connector.client.thin.tcp`
+Register name: `client.connector`
 
 
 [cols="2,1,3",opts="header"]
@@ -303,6 +303,11 @@ Register name: `connector.client.thin.tcp`
 |ActiveSessionsCount|   integer|   Active TCP sessions count.
 |ReceivedBytesCount|   long|   Received bytes count.
 |RejectedSslSessionsCount|   integer|   TCP sessions count that were rejected due to the SSL errors (metric is exported only if SSL is enabled).
+|RejectedSessionsTimeout|   integer|   TCP sessions count that were rejected due to handshake timeout.
+|RejectedSessionsAuthenticationFailed|   integer|   TCP sessions count that were rejected due to failed authentication.
+|RejectedSessionsTotal|   integer|   Total number of rejected TCP connections.
+|{clientType}.AcceptedSessions|   integer|   Number of successfully established sessions for the client type.
+|{clientType}.ActiveSessions|   integer|   Number of active sessions for the client type.
 |SentBytesCount|   long|   Sent bytes count.
 |SslEnabled|   boolean|   Indicates whether SSL is enabled.
 |SslHandshakeDurationHistogram|   histogram|   Histogram of SSL handshake duration in milliseconds (metric is exported only if SSL is enabled).
@@ -311,7 +316,7 @@ Register name: `connector.client.thin.tcp`
 
 == Ignite REST Client Connector
 
-Register name: `connector.client.rest.tcp`
+Register name: `rest.client`
 
 
 [cols="2,1,3",opts="header"]
diff --git a/modules/clients/src/test/java/org/apache/ignite/common/NodeSslConnectionMetricTest.java b/modules/clients/src/test/java/org/apache/ignite/common/NodeSslConnectionMetricTest.java
index 99b1904..149be86 100644
--- a/modules/clients/src/test/java/org/apache/ignite/common/NodeSslConnectionMetricTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/common/NodeSslConnectionMetricTest.java
@@ -49,7 +49,7 @@ import static java.sql.DriverManager.getConnection;
 import static org.apache.ignite.Ignition.startClient;
 import static org.apache.ignite.internal.client.GridClientFactory.start;
 import static org.apache.ignite.internal.managers.discovery.GridDiscoveryManager.DISCO_METRICS;
-import static org.apache.ignite.internal.processors.odbc.ClientListenerProcessor.CLIENT_CONNECTOR_METRIC_REGISTRY_NAME;
+import static org.apache.ignite.internal.processors.metric.GridMetricManager.CLIENT_CONNECTOR_METRICS;
 import static org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestProtocol.REST_CONNECTOR_METRIC_REGISTRY_NAME;
 import static org.apache.ignite.internal.util.nio.GridNioServer.RECEIVED_BYTES_METRIC_NAME;
 import static org.apache.ignite.internal.util.nio.GridNioServer.SENT_BYTES_METRIC_NAME;
@@ -103,7 +103,7 @@ public class NodeSslConnectionMetricTest extends GridCommonAbstractTest {
         assertNull(commReg.<HistogramMetric>findMetric(SSL_HANDSHAKE_DURATION_HISTOGRAM_METRIC_NAME));
         assertEquals(0, commReg.<IntMetric>findMetric(SESSIONS_CNT_METRIC_NAME).value());
 
-        MetricRegistry cliConnReg = mreg(srv, CLIENT_CONNECTOR_METRIC_REGISTRY_NAME);
+        MetricRegistry cliConnReg = mreg(srv, CLIENT_CONNECTOR_METRICS);
 
         assertFalse(cliConnReg.<BooleanMetric>findMetric(SSL_ENABLED_METRIC_NAME).value());
         assertNull(cliConnReg.<IntMetric>findMetric(SSL_REJECTED_SESSIONS_CNT_METRIC_NAME));
@@ -130,7 +130,7 @@ public class NodeSslConnectionMetricTest extends GridCommonAbstractTest {
     /** Tests SSL metrics produced by JDBC connection. */
     @Test
     public void testJdbc() throws Exception {
-        MetricRegistry reg = mreg(startClusterNode(0), CLIENT_CONNECTOR_METRIC_REGISTRY_NAME);
+        MetricRegistry reg = mreg(startClusterNode(0), CLIENT_CONNECTOR_METRICS);
 
         assertEquals(0, reg.<LongMetric>findMetric(SENT_BYTES_METRIC_NAME).value());
         assertEquals(0, reg.<LongMetric>findMetric(RECEIVED_BYTES_METRIC_NAME).value());
@@ -276,7 +276,7 @@ public class NodeSslConnectionMetricTest extends GridCommonAbstractTest {
     /** Tests SSL metrics produced by thin client connection. */
     @Test
     public void testClientConnector() throws Exception {
-        MetricRegistry reg = mreg(startClusterNode(0), CLIENT_CONNECTOR_METRIC_REGISTRY_NAME);
+        MetricRegistry reg = mreg(startClusterNode(0), CLIENT_CONNECTOR_METRICS);
 
         assertEquals(0, reg.<LongMetric>findMetric(SENT_BYTES_METRIC_NAME).value());
         assertEquals(0, reg.<LongMetric>findMetric(RECEIVED_BYTES_METRIC_NAME).value());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
index 48c9aef..25f0f78 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
@@ -144,6 +144,9 @@ public class GridMetricManager extends GridManagerAdapter<MetricExporterSpi> imp
     /** Cluster metrics prefix. */
     public static final String CLUSTER_METRICS = "cluster";
 
+    /** Client metrics prefix. */
+    public static final String CLIENT_CONNECTOR_METRICS = metricName("client", "connector");
+
     /** Transaction metrics prefix. */
     public static final String TX_METRICS = "tx";
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java
index a541a05..6143265 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java
@@ -28,6 +28,11 @@ import org.jetbrains.annotations.Nullable;
  */
 public interface ClientListenerConnectionContext {
     /**
+     * @return Client type.
+     */
+    byte clientType();
+
+    /**
      * @return Current connection id.
      */
     long connectionId();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerMetrics.java
new file mode 100644
index 0000000..ee96156
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerMetrics.java
@@ -0,0 +1,155 @@
+/*
+ * 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.processors.odbc;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils;
+
+import static org.apache.ignite.internal.processors.metric.GridMetricManager.CLIENT_CONNECTOR_METRICS;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.JDBC_CLIENT;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.ODBC_CLIENT;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.THIN_CLIENT;
+
+/**
+ * Client listener metrics.
+ */
+public class ClientListenerMetrics {
+    /** Handshakes rejected by timeout metric label. */
+    public static final String METRIC_REJECTED_TIMEOUT = "RejectedSessionsTimeout";
+
+    /** Handshakes rejected by authentication metric label. */
+    public static final String METRIC_REJECTED_AUTHENTICATION = "RejectedSessionsAuthenticationFailed";
+
+    /** Total number of rejected handshakes. */
+    public static final String METRIC_REJECTED_TOTAL = "RejectedSessionsTotal";
+
+    /** Number of successfully established sessions. */
+    public static final String METRIC_ACEPTED = "AcceptedSessions";
+
+    /** Number of active sessions. */
+    public static final String METRIC_ACTIVE = "ActiveSessions";
+
+    /** Rejected by timeout. */
+    private final IntMetricImpl rejectedTimeout;
+
+    /** Rejected by authentication. */
+    private final IntMetricImpl rejectedAuth;
+
+    /** Total number of rejected connections. */
+    private final IntMetricImpl rejectedTotal;
+
+    /** Connections accepted. */
+    private final IntMetricImpl[] accepted;
+
+    /** Number of active connections. */
+    private final IntMetricImpl[] active;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public ClientListenerMetrics(GridKernalContext ctx) {
+        MetricRegistry mreg = ctx.metric().registry(CLIENT_CONNECTOR_METRICS);
+
+        rejectedTimeout = mreg.intMetric(METRIC_REJECTED_TIMEOUT,
+                "TCP sessions count that were rejected due to handshake timeout.");
+
+        rejectedAuth = mreg.intMetric(METRIC_REJECTED_AUTHENTICATION,
+                "TCP sessions count that were rejected due to failed authentication.");
+
+        rejectedTotal = mreg.intMetric(METRIC_REJECTED_TOTAL, "Total number of rejected TCP connections.");
+
+        final byte[] supportedClients = { ODBC_CLIENT, JDBC_CLIENT, THIN_CLIENT };
+        accepted = new IntMetricImpl[supportedClients.length];
+        active = new IntMetricImpl[supportedClients.length];
+
+        for (byte clientType : supportedClients) {
+            String clientLabel = clientTypeLabel(clientType);
+
+            String labelAccepted = MetricUtils.metricName(clientLabel, METRIC_ACEPTED);
+            accepted[clientType] = mreg.intMetric(labelAccepted,
+                    "Number of successfully established sessions for the client type.");
+
+            String labelActive = MetricUtils.metricName(clientLabel, METRIC_ACTIVE);
+            active[clientType] = mreg.intMetric(labelActive, "Number of active sessions for the client type.");
+        }
+    }
+
+    /**
+     * Callback invoked when handshake is timed out.
+     */
+    public void onHandshakeTimeout() {
+        rejectedTimeout.increment();
+        rejectedTotal.increment();
+    }
+
+    /**
+     * Callback invoked when authentication is failed.
+     */
+    public void onFailedAuth() {
+        rejectedAuth.increment();
+        rejectedTotal.increment();
+    }
+
+    /**
+     * Callback invoked when handshake is rejected.
+     */
+    public void onGeneralReject() {
+        rejectedTotal.increment();
+    }
+
+    /**
+     * Callback invoked when handshake is accepted.
+     *
+     * @param clientType Client type.
+     */
+    public void onHandshakeAccept(byte clientType) {
+        accepted[clientType].increment();
+        active[clientType].increment();
+    }
+
+    /**
+     * Callback invoked when client is disconnected.
+     *
+     * @param clientType Client type.
+     */
+    public void onDisconnect(byte clientType) {
+        active[clientType].add(-1);
+    }
+
+    /**
+     * Get label for a client.
+     * @param clientType Client type.
+     * @return Label for a client.
+     */
+    private String clientTypeLabel(byte clientType) {
+        switch (clientType) {
+            case ODBC_CLIENT:
+                return "odbc";
+
+            case JDBC_CLIENT:
+                return "jdbc";
+
+            case THIN_CLIENT:
+                return "thin";
+
+            default:
+                return "unknown";
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
index 92aec47..e50cbf7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
@@ -84,11 +84,14 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
     private final IgniteLogger log;
 
     /** Client connection config. */
-    private ClientConnectorConfiguration cliConnCfg;
+    private final ClientConnectorConfiguration cliConnCfg;
 
     /** Thin client configuration. */
     private final ThinClientConfiguration thinCfg;
 
+    /** Metrics. */
+    private final ClientListenerMetrics metrics;
+
     /**
      * Constructor.
      *
@@ -109,6 +112,8 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
 
         thinCfg = cliConnCfg.getThinClientConfiguration() == null ? new ThinClientConfiguration()
             : new ThinClientConfiguration(cliConnCfg.getThinClientConfiguration());
+
+        metrics = new ClientListenerMetrics(ctx);
     }
 
     /** {@inheritDoc} */
@@ -126,9 +131,12 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
     @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
         ClientListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY);
 
-        if (connCtx != null)
+        if (connCtx != null) {
             connCtx.onDisconnected();
 
+            metrics.onDisconnect(connCtx.clientType());
+        }
+
         if (log.isDebugEnabled()) {
             if (e == null)
                 log.debug("Client disconnected: " + ses.remoteAddress());
@@ -247,6 +255,8 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
             @Override public void run() {
                 ses.close();
 
+                metrics.onHandshakeTimeout();
+
                 U.warn(log, "Unable to perform handshake within timeout " +
                     "[timeout=" + handshakeTimeout + ", remoteAddr=" + ses.remoteAddress() + ']');
             }
@@ -326,8 +336,12 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
             cancelHandshakeTimeout(ses);
 
             connCtx.handler().writeHandshake(writer);
+
+            metrics.onHandshakeAccept(clientType);
         }
         catch (IgniteAccessControlException authEx) {
+            metrics.onFailedAuth();
+
             writer.writeBoolean(false);
 
             writer.writeShort((short)0);
@@ -342,6 +356,8 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<Clie
         catch (IgniteCheckedException e) {
             U.warn(log, "Error during handshake [rmtAddr=" + ses.remoteAddress() + ", msg=" + e.getMessage() + ']');
 
+            metrics.onGeneralReject();
+
             ClientListenerProtocolVersion currVer;
 
             if (connCtx == null)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
index cce21d2..6621b80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
@@ -59,6 +59,7 @@ import org.apache.ignite.thread.OomExceptionHandler;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.metric.GridMetricManager.CLIENT_CONNECTOR_METRICS;
 import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.CONN_CTX_META_KEY;
 
@@ -72,9 +73,6 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
     /** */
     public static final String CLI_CONN_VIEW_DESC = "Client connections";
 
-    /** The name of the metric registry associated with the thin client connector. */
-    public static final String CLIENT_CONNECTOR_METRIC_REGISTRY_NAME = metricName("connector", "client", "thin", "tcp");
-
     /** Default client connector configuration. */
     public static final ClientConnectorConfiguration DFLT_CLI_CFG = new ClientConnectorConfigurationEx();
 
@@ -84,9 +82,6 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
     /** Cancel counter. For testing purposes only. */
     public static final AtomicLong CANCEL_COUNTER = new AtomicLong(0);
 
-    /** Default number of selectors. */
-    private static final int DFLT_SELECTOR_CNT = Math.min(4, Runtime.getRuntime().availableProcessors());
-
     /** Default TCP direct buffer flag. */
     private static final boolean DFLT_TCP_DIRECT_BUF = true;
 
@@ -180,7 +175,7 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
                             .filters(filters)
                             .directMode(true)
                             .idleTimeout(idleTimeout > 0 ? idleTimeout : Long.MAX_VALUE)
-                            .metricRegistry(ctx.metric().registry(CLIENT_CONNECTOR_METRIC_REGISTRY_NAME))
+                            .metricRegistry(ctx.metric().registry(CLIENT_CONNECTOR_METRICS))
                             .build();
 
                         ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass());
@@ -338,7 +333,7 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
                     "(SSL is enabled but factory is null). Check the ClientConnectorConfiguration");
 
             GridNioSslFilter sslFilter = new GridNioSslFilter(sslCtxFactory.create(),
-                true, ByteOrder.nativeOrder(), log, ctx.metric().registry(CLIENT_CONNECTOR_METRIC_REGISTRY_NAME));
+                true, ByteOrder.nativeOrder(), log, ctx.metric().registry(CLIENT_CONNECTOR_METRICS));
 
             sslFilter.directMode(true);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index a798ed6..cfe4263 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.F;
 
 import static org.apache.ignite.internal.jdbc.thin.JdbcThinUtils.nullableBooleanFromByte;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.JDBC_CLIENT;
 
 /**
  * JDBC Connection Context.
@@ -83,10 +84,10 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
     private final int maxCursors;
 
     /** Message parser. */
-    private JdbcMessageParser parser = null;
+    private JdbcMessageParser parser;
 
     /** Request handler. */
-    private JdbcRequestHandler handler = null;
+    private JdbcRequestHandler handler;
 
     /** Current protocol context. */
     private JdbcProtocolContext protoCtx;
@@ -125,6 +126,11 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
     }
 
     /** {@inheritDoc} */
+    @Override public byte clientType() {
+        return JDBC_CLIENT;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isVersionSupported(ClientListenerProtocolVersion ver) {
         return SUPPORTED_VERS.contains(ver);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
index 8c8d6b7..16721be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
@@ -33,6 +33,8 @@ import org.apache.ignite.internal.processors.query.NestedTxMode;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 
+import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.ODBC_CLIENT;
+
 /**
  * ODBC Connection Context.
  */
@@ -71,10 +73,10 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte
     private final int maxCursors;
 
     /** Message parser. */
-    private OdbcMessageParser parser = null;
+    private OdbcMessageParser parser;
 
     /** Request handler. */
-    private OdbcRequestHandler handler = null;
+    private OdbcRequestHandler handler;
 
     /** Logger. */
     private final IgniteLogger log;
@@ -107,6 +109,11 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte
     }
 
     /** {@inheritDoc} */
+    @Override public byte clientType() {
+        return ODBC_CLIENT;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isVersionSupported(ClientListenerProtocolVersion ver) {
         return SUPPORTED_VERS.contains(ver);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index de346f4..eb765bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 
+import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.THIN_CLIENT;
 import static org.apache.ignite.internal.processors.platform.client.ClientBitmaskFeature.USER_ATTRIBUTES;
 import static org.apache.ignite.internal.processors.platform.client.ClientProtocolVersionFeature.AUTHORIZATION;
 import static org.apache.ignite.internal.processors.platform.client.ClientProtocolVersionFeature.BITMAP_FEATURES;
@@ -167,6 +168,11 @@ public class ClientConnectionContext extends ClientListenerAbstractConnectionCon
     }
 
     /** {@inheritDoc} */
+    @Override public byte clientType() {
+        return THIN_CLIENT;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isVersionSupported(ClientListenerProtocolVersion ver) {
         return SUPPORTED_VERS.contains(ver);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 081e461..8cb7197 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -66,7 +66,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
     private GridTcpRestNioListener lsnr;
 
     /** The name of the metric registry associated with the REST TCP connector. */
-    public static final String REST_CONNECTOR_METRIC_REGISTRY_NAME = metricName("connector", "client", "rest", "tcp");
+    public static final String REST_CONNECTOR_METRIC_REGISTRY_NAME = metricName("rest", "client");
 
     /** @param ctx Context. */
     public GridTcpRestProtocol(GridKernalContext ctx) {
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/ClientListenerMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/ClientListenerMetricsTest.java
new file mode 100644
index 0000000..c2db058
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/ClientListenerMetricsTest.java
@@ -0,0 +1,221 @@
+/*
+ * 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.processors.odbc;
+
+import org.apache.ignite.Ignition;
+import org.apache.ignite.client.ClientAuthenticationException;
+import org.apache.ignite.client.ClientException;
+import org.apache.ignite.client.Config;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.client.SslMode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.spi.metric.IntMetric;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.metric.GridMetricManager.CLIENT_CONNECTOR_METRICS;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerMetrics.METRIC_ACEPTED;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerMetrics.METRIC_ACTIVE;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerMetrics.METRIC_REJECTED_AUTHENTICATION;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerMetrics.METRIC_REJECTED_TIMEOUT;
+import static org.apache.ignite.internal.processors.odbc.ClientListenerMetrics.METRIC_REJECTED_TOTAL;
+import static org.apache.ignite.ssl.SslContextFactory.DFLT_STORE_TYPE;
+
+/**
+ * Client listener metrics tests.
+ */
+public class ClientListenerMetricsTest extends GridCommonAbstractTest {
+    /**
+     * Check that valid connections and disconnections to the grid affect metrics.
+     */
+    @Test
+    public void testClientListenerMetricsAccept() throws Exception {
+        try (IgniteEx ignite = startGrid(0))
+        {
+            MetricRegistry mreg = ignite.context().metric().registry(CLIENT_CONNECTOR_METRICS);
+
+            checkConnectionsMetrics(mreg, 0, 0);
+
+            IgniteClient client0 = Ignition.startClient(getClientConfiguration());
+
+            checkConnectionsMetrics(mreg, 1, 1);
+
+            client0.close();
+
+            checkConnectionsMetrics(mreg, 1, 0);
+
+            IgniteClient client1 = Ignition.startClient(getClientConfiguration());
+
+            checkConnectionsMetrics(mreg, 2, 1);
+
+            IgniteClient client2 = Ignition.startClient(getClientConfiguration());
+
+            checkConnectionsMetrics(mreg, 3, 2);
+
+            client1.close();
+
+            checkConnectionsMetrics(mreg, 3, 1);
+
+            client2.close();
+
+            checkConnectionsMetrics(mreg, 3, 0);
+        }
+    }
+
+    /**
+     * Check that failed connection attempts to the grid affect metrics.
+     */
+    @Test
+    public void testClientListenerMetricsReject() throws Exception {
+        cleanPersistenceDir();
+
+        IgniteConfiguration nodeCfg = getConfiguration()
+            .setClientConnectorConfiguration(new ClientConnectorConfiguration()
+                .setHandshakeTimeout(2000))
+            .setAuthenticationEnabled(true)
+            .setDataStorageConfiguration(new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
+                    .setPersistenceEnabled(true)));
+
+        try (IgniteEx ignite = startGrid(nodeCfg))
+        {
+            ignite.cluster().state(ClusterState.ACTIVE);
+            MetricRegistry mreg = ignite.context().metric().registry(CLIENT_CONNECTOR_METRICS);
+
+            checkRejectMetrics(mreg, 0, 0, 0);
+
+            ClientConfiguration cfgSsl = getClientConfiguration()
+                .setSslMode(SslMode.REQUIRED)
+                .setSslClientCertificateKeyStorePath(GridTestUtils.keyStorePath("client"))
+                .setSslClientCertificateKeyStoreType(DFLT_STORE_TYPE)
+                .setSslClientCertificateKeyStorePassword("123456")
+                .setSslTrustCertificateKeyStorePath(GridTestUtils.keyStorePath("trustone"))
+                .setSslTrustCertificateKeyStoreType(DFLT_STORE_TYPE)
+                .setSslTrustCertificateKeyStorePassword("123456");
+
+            GridTestUtils.assertThrows(log, () -> {
+                Ignition.startClient(cfgSsl);
+                return null;
+            }, ClientException.class, null);
+
+            checkRejectMetrics(mreg, 1, 0, 1);
+
+            ClientConfiguration cfgAuth = getClientConfiguration()
+                .setUserName("SomeRandomInvalidName")
+                .setUserPassword("42");
+
+            GridTestUtils.assertThrows(log, () -> {
+                Ignition.startClient(cfgAuth);
+                return null;
+            }, ClientAuthenticationException.class, null);
+
+            checkRejectMetrics(mreg, 1, 1, 2);
+        }
+    }
+
+    /**
+     * Check that failed connection attempts to the grid affect metrics.
+     */
+    @Test
+    public void testClientListenerMetricsRejectGeneral() throws Exception {
+        IgniteConfiguration nodeCfg = getConfiguration()
+            .setClientConnectorConfiguration(new ClientConnectorConfiguration()
+            .setThinClientEnabled(false));
+
+        try (IgniteEx ignite = startGrid(nodeCfg))
+        {
+            MetricRegistry mreg = ignite.context().metric().registry(CLIENT_CONNECTOR_METRICS);
+
+            checkRejectMetrics(mreg, 0, 0, 0);
+
+            GridTestUtils.assertThrows(log, () -> {
+                Ignition.startClient(getClientConfiguration());
+                return null;
+            }, RuntimeException.class, "Thin client connection is not allowed");
+
+            checkRejectMetrics(mreg, 0, 0, 1);
+        }
+    }
+
+    /** */
+    private static ClientConfiguration getClientConfiguration() {
+        return new ClientConfiguration()
+                .setAddresses(Config.SERVER)
+                .setSendBufferSize(0)
+                .setReceiveBufferSize(0);
+    }
+
+    /**
+     * Wait for specific metric to change
+     * @param mreg Metric registry.
+     * @param metric Metric to check.
+     * @param value Metric value to wait for.
+     * @param timeout Timeout.
+     */
+    private void waitForMetricValue(MetricRegistry mreg, String metric, long value, long timeout)
+        throws IgniteInterruptedCheckedException {
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return mreg.<IntMetric>findMetric(metric).value() == value;
+            }
+        }, timeout);
+        assertEquals(mreg.<IntMetric>findMetric(metric).value(), value);
+    }
+
+    /**
+     * Check client metrics
+     * @param mreg Client metric registry
+     * @param rejectedTimeout Expected number of connection attepmts rejected by timeout.
+     * @param rejectedAuth Expected number of connection attepmts rejected because of failed authentication.
+     * @param rejectedTotal Expected number of connection attepmts rejected in total.
+     */
+    private void checkRejectMetrics(MetricRegistry mreg, int rejectedTimeout, int rejectedAuth, int rejectedTotal)
+        throws IgniteInterruptedCheckedException {
+        waitForMetricValue(mreg, METRIC_REJECTED_TOTAL, rejectedTotal, 10_000);
+        assertEquals(rejectedTimeout, mreg.<IntMetric>findMetric(METRIC_REJECTED_TIMEOUT).value());
+        assertEquals(rejectedAuth, mreg.<IntMetric>findMetric(METRIC_REJECTED_AUTHENTICATION).value());
+        assertEquals(0, mreg.<IntMetric>findMetric(MetricUtils.metricName("thin", METRIC_ACEPTED)).value());
+        assertEquals(0, mreg.<IntMetric>findMetric(MetricUtils.metricName("thin", METRIC_ACTIVE)).value());
+    }
+
+    /**
+     * Check client metrics
+     * @param mreg Client metric registry
+     * @param accepted Expected number of accepted connections.
+     * @param active Expected number of active connections.
+     */
+    private void checkConnectionsMetrics(MetricRegistry mreg, int accepted, int active)
+        throws IgniteInterruptedCheckedException {
+        waitForMetricValue(mreg, MetricUtils.metricName("thin", METRIC_ACTIVE), active, 10_000);
+        assertEquals(accepted, mreg.<IntMetric>findMetric(MetricUtils.metricName("thin", METRIC_ACEPTED)).value());
+        assertEquals(0, mreg.<IntMetric>findMetric(METRIC_REJECTED_TIMEOUT).value());
+        assertEquals(0, mreg.<IntMetric>findMetric(METRIC_REJECTED_AUTHENTICATION).value());
+        assertEquals(0, mreg.<IntMetric>findMetric(METRIC_REJECTED_TOTAL).value());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 62dd926..4039380 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -110,6 +110,7 @@ import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorageT
 import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageHistoryCacheTest;
 import org.apache.ignite.internal.processors.metastorage.persistence.DmsDataWriterWorkerTest;
 import org.apache.ignite.internal.processors.metastorage.persistence.InMemoryCachedDistributedMetaStorageBridgeTest;
+import org.apache.ignite.internal.processors.odbc.ClientListenerMetricsTest;
 import org.apache.ignite.internal.processors.odbc.OdbcConfigurationValidationSelfTest;
 import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerUtilsTest;
@@ -208,6 +209,8 @@ import org.junit.runners.Suite;
 
     IgniteExceptionInNioWorkerSelfTest.class,
     IgniteLocalNodeMapBeforeStartTest.class,
+
+    ClientListenerMetricsTest.class,
     OdbcConfigurationValidationSelfTest.class,
     OdbcEscapeSequenceSelfTest.class,
     SqlListenerUtilsTest.class,
diff --git a/modules/platforms/cpp/thin-client-test/include/test_utils.h b/modules/platforms/cpp/thin-client-test/include/test_utils.h
index e0f0ff5..7f9068a 100644
--- a/modules/platforms/cpp/thin-client-test/include/test_utils.h
+++ b/modules/platforms/cpp/thin-client-test/include/test_utils.h
@@ -20,6 +20,9 @@
 
 #include <string>
 
+#include <boost/chrono.hpp>
+#include <boost/thread.hpp>
+
 #include <ignite/ignition.h>
 
 #define MUTE_TEST_FOR_TEAMCITY \
@@ -83,6 +86,33 @@ namespace ignite_test
      * @return Number of occurrences.
      */
     size_t GetLineOccurrencesInFile(const std::string& filePath, const std::string& line);
+
+    /**
+     * Wait for condition.
+     * @tparam T Type of condition function.
+     * @param func Function that should check for condition and return true once it's performed.
+     * @param timeout Timeout to wait.
+     * @return True if condition was met, false if timeout has been reached.
+     */
+    template<typename F>
+    bool WaitForCondition(F func, int32_t timeout)
+    {
+        using namespace boost::chrono;
+
+        const int32_t span = 200;
+
+        steady_clock::time_point begin = steady_clock::now();
+
+        while (!func())
+        {
+            boost::this_thread::sleep_for(milliseconds(span));
+
+            if (timeout && duration_cast<milliseconds>(steady_clock::now() - begin).count() >= timeout)
+                return func();
+        }
+
+        return true;
+    }
 }
 
 #endif // _IGNITE_THIN_CLIENT_TEST_TEST_UTILS
\ No newline at end of file
diff --git a/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp b/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp
index bb50b16..7476305 100644
--- a/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp
+++ b/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp
@@ -16,6 +16,7 @@
  */
 
 #include <boost/test/unit_test.hpp>
+#include <boost/bind.hpp>
 
 #include <ignite/ignition.h>
 
@@ -41,6 +42,17 @@ public:
     }
 
     /**
+     * Wait for connections.
+     * @return True if condition was met, false if timeout has been reached.
+     */
+    bool WaitForConnections(size_t expected, int32_t timeout = 5000)
+    {
+        return ignite_test::WaitForCondition(
+                boost::bind(&IgniteClientTestSuiteFixture::CheckActiveConnections, this, expected),
+                timeout);
+    }
+
+    /**
      * Check that if client started with given configuration and connection limit then the actual number of active
      * connections is equal to the expected value.
      *
@@ -53,10 +65,22 @@ public:
         cfg.SetConnectionsLimit(limit);
         IgniteClient client = IgniteClient::Start(cfg);
 
+        BOOST_CHECK(WaitForConnections(expect));
         BOOST_CHECK_EQUAL(GetActiveConnections(), expect);
     }
 
     /**
+     * Check number of active connections.
+     *
+     * @param expect connections to expect.
+     * @return @c true on success.
+     */
+    bool CheckActiveConnections(size_t expect)
+    {
+        return GetActiveConnections() == expect;
+    }
+
+    /**
      * Get Number of active connections.
      *
      * @return Number of active connections.