You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2020/07/16 10:51:56 UTC

[ignite] branch master updated: IGNITE-13060 Tracing: initial implementation - Fixes #7976.

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

irakov 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 0ef1deb  IGNITE-13060 Tracing: initial implementation - Fixes #7976.
0ef1deb is described below

commit 0ef1debd2fc9452376a9e1ce36f0a9a945469783
Author: alapin <la...@gmail.com>
AuthorDate: Thu Jul 16 13:44:37 2020 +0300

    IGNITE-13060 Tracing: initial implementation - Fixes #7976.
    
    Signed-off-by: Ivan Rakov <iv...@gmail.com>
---
 .../ignite/internal/commandline/CommandList.java   |    5 +-
 .../commandline/TracingConfigurationCommand.java   |  346 ++++++
 .../TracingConfigurationArguments.java             |  180 ++++
 .../TracingConfigurationCommandArg.java            |   79 ++
 .../TracingConfigurationSubcommand.java            |  104 ++
 .../commandline/CommandHandlerParsingTest.java     |  191 ++++
 .../testsuites/IgniteControlUtilityTestSuite.java  |    3 +
 .../util/GridCommandHandlerClusterByClassTest.java |    3 +
 ...GridCommandHandlerTracingConfigurationTest.java |  390 +++++++
 .../src/main/java/org/apache/ignite/Ignite.java    |   17 +
 .../java/org/apache/ignite/IgniteTransactions.java |    7 +
 .../ignite/configuration/IgniteConfiguration.java  |   26 +
 .../org/apache/ignite/events/DiscoveryEvent.java   |   59 ++
 .../apache/ignite/internal/GridKernalContext.java  |    8 +
 .../ignite/internal/GridKernalContextImpl.java     |   13 +
 .../ignite/internal/IgniteComponentType.java       |    8 +
 .../org/apache/ignite/internal/IgniteFeatures.java |    3 +
 .../org/apache/ignite/internal/IgniteKernal.java   |   26 +-
 .../org/apache/ignite/internal/IgnitionEx.java     |    4 +
 .../managers/communication/GridIoManager.java      |  150 ++-
 .../managers/communication/GridIoMessage.java      |   41 +-
 .../communication/GridIoSecurityAwareMessage.java  |    6 +-
 .../managers/communication/TraceRunnable.java      |   63 ++
 .../managers/discovery/GridDiscoveryManager.java   |  268 +++--
 .../managers/tracing/GridTracingManager.java       |  534 ++++++++++
 .../cache/CacheAffinitySharedManager.java          |    7 +
 .../processors/cache/GridCacheAdapter.java         |    9 +-
 .../cache/GridCachePartitionExchangeManager.java   |   24 +
 .../processors/cache/GridCacheProcessor.java       |    2 +-
 .../cache/distributed/dht/GridDhtLockFuture.java   |   67 +-
 .../distributed/dht/GridDhtTxFinishFuture.java     |  106 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java     |   19 +
 .../distributed/dht/GridDhtTxPrepareFuture.java    |  205 ++--
 .../dht/colocated/GridDhtColocatedLockFuture.java  |  148 +--
 .../preloader/GridDhtPartitionsExchangeFuture.java |   47 +
 ...dNearOptimisticSerializableTxPrepareFuture.java |   44 +-
 .../near/GridNearOptimisticTxPrepareFuture.java    |  115 +-
 .../GridNearOptimisticTxPrepareFutureAdapter.java  |   54 +-
 .../near/GridNearPessimisticTxPrepareFuture.java   |   70 +-
 .../distributed/near/GridNearTxFinishFuture.java   |  133 +--
 .../cache/distributed/near/GridNearTxLocal.java    |  769 +++++++-------
 .../internal/processors/cache/mvcc/MvccUtils.java  |    3 +-
 .../wal/reader/StandaloneGridKernalContext.java    |    7 +
 .../cache/transactions/IgniteTransactionsImpl.java |   32 +-
 .../cache/transactions/IgniteTxHandler.java        |  572 +++++-----
 .../cache/transactions/IgniteTxManager.java        |    6 +-
 .../cache/transactions/TransactionProxyImpl.java   |  153 ++-
 .../continuous/GridContinuousProcessor.java        |    2 +-
 .../platform/client/tx/ClientTxStartRequest.java   |    3 +-
 .../internal/processors/tracing/DeferredSpan.java  |   91 ++
 .../ignite/internal/processors/tracing/MTC.java    |  118 +++
 .../internal/processors/tracing/NoopSpan.java      |   74 ++
 .../internal/processors/tracing/NoopTracing.java   |   76 ++
 .../ignite/internal/processors/tracing/Span.java   |   85 ++
 .../internal/processors/tracing/SpanImpl.java      |  102 ++
 .../internal/processors/tracing/SpanManager.java   |   73 ++
 .../internal/processors/tracing/SpanTags.java      |   97 ++
 .../internal/processors/tracing/SpanType.java      |  254 +++++
 .../internal/processors/tracing/Tracing.java       |   45 +
 .../DistributedTracingConfiguration.java           |   49 +
 .../GridTracingConfigurationManager.java           |  214 ++++
 .../NoopTracingConfigurationManager.java           |   67 ++
 .../processors/tracing/messages/SpanContainer.java |   82 ++
 .../tracing/messages/SpanTransport.java}           |   31 +-
 .../tracing/messages/TraceableMessage.java}        |   28 +-
 .../tracing/messages/TraceableMessagesHandler.java |  106 ++
 .../tracing/messages/TraceableMessagesTable.java   |   88 ++
 .../tracing/TracingSpiType.java}                   |   36 +-
 .../apache/ignite/internal/util/IgniteUtils.java   |   12 +
 .../ignite/internal/util/nio/GridNioServer.java    |  224 ++--
 .../internal/util/nio/GridNioTracerFilter.java     |  125 +++
 .../util/nio/GridSelectorNioSessionImpl.java       |    6 +
 .../internal/util/nio/SessionWriteRequest.java     |    8 +
 .../VisorTracingConfigurationItem.java             |  177 ++++
 .../VisorTracingConfigurationOperation.java        |   56 +
 .../VisorTracingConfigurationTask.java             |  204 ++++
 .../VisorTracingConfigurationTaskArg.java          |  103 ++
 .../VisorTracingConfigurationTaskResult.java       |  104 ++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java    |    5 +-
 .../spi/communication/tcp/TcpCommunicationSpi.java |   19 +
 .../spi/discovery/DiscoveryNotification.java       |  138 +++
 .../ignite/spi/discovery/DiscoverySpiListener.java |   21 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java       |  112 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java       |  345 +++++-
 .../ignite/spi/discovery/tcp/TcpDiscoveryImpl.java |   11 +
 .../TcpDiscoveryAbstractTraceableMessage.java      |   72 ++
 .../messages/TcpDiscoveryCustomEventMessage.java   |    2 +-
 .../messages/TcpDiscoveryJoinRequestMessage.java   |    2 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java        |    2 +-
 .../tcp/messages/TcpDiscoveryNodeAddedMessage.java |    2 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java    |    2 +-
 .../tcp/messages/TcpDiscoveryNodeLeftMessage.java  |    2 +-
 .../NoopSpiSpecificSpan.java}                      |   46 +-
 .../apache/ignite/spi/tracing/NoopTracingSpi.java  |   68 ++
 .../java/org/apache/ignite/spi/tracing/Scope.java  |   80 ++
 .../org/apache/ignite/spi/tracing/SpanStatus.java  |   30 +
 .../SpiSpecificSpan.java}                          |   50 +-
 .../tracing/TracingConfigurationCoordinates.java   |  145 +++
 .../spi/tracing/TracingConfigurationManager.java   |  159 +++
 .../tracing/TracingConfigurationParameters.java    |  157 +++
 .../org/apache/ignite/spi/tracing/TracingSpi.java  |   59 ++
 .../apache/ignite/spi/tracing/package-info.java    |   22 +
 .../main/resources/META-INF/classnames.properties  |    2 +
 ...IgniteMarshallerCacheClassNameConflictTest.java |   17 +-
 .../cache/IgniteMarshallerCacheFSRestoreTest.java  |   17 +-
 .../IgniteAbstractStandByClientReconnectTest.java  |   14 +-
 .../TxDataConsistencyOnCommitFailureTest.java      |    4 +-
 .../internal/processors/igfs/IgfsIgniteMock.java   |    8 +
 .../spi/discovery/AbstractDiscoverySelfTest.java   |   28 +-
 .../discovery/LongClientConnectToClusterTest.java  |   17 +-
 .../apache/ignite/testframework/GridTestUtils.java |   17 +-
 .../ignite/testframework/junits/IgniteMock.java    |    8 +
 .../junits/multijvm/IgniteProcessProxy.java        |    7 +
 .../ignite/testsuites/IgniteBasicTestSuite.java    |    2 +-
 ...ridCommandHandlerClusterByClassTest_help.output |   18 +
 ...andHandlerClusterByClassWithSSLTest_help.output |   18 +
 .../processors/cache/jta/CacheJtaManager.java      |    3 +-
 modules/opencensus/pom.xml                         |    6 +
 .../exporter/trace/TimeLimitedHandler.java         |  146 +++
 .../tracing/opencensus/OpenCensusSpanAdapter.java  |   82 ++
 .../opencensus/OpenCensusTraceExporter.java        |   68 ++
 .../tracing/opencensus/OpenCensusTracingSpi.java   |  155 +++
 .../spi/tracing/opencensus/StatusMatchTable.java   |   55 +
 .../spi/tracing/opencensus/package-info.java       |   22 +
 .../ignite/TracingConfigurationValidationTest.java |  125 +++
 .../monitoring/opencensus/AbstractTracingTest.java |  369 +++++++
 .../opencensus/IgniteOpenCensusSuite.java          |   41 +
 .../monitoring/opencensus/MixedTracingSpiTest.java |  126 +++
 .../OpenCensusTracingConfigurationGetAllTest.java  |  106 ++
 .../OpenCensusTracingConfigurationGetTest.java     |  149 +++
 ...OpenCensusTracingConfigurationResetAllTest.java |   98 ++
 .../OpenCensusTracingConfigurationResetTest.java   |  154 +++
 .../opencensus/OpenCensusTracingSpiTest.java       |  372 +++++++
 .../OpenCensusTxTracingConfigurationTest.java      |  294 ++++++
 .../opencensus/OpenCensusTxTracingTest.java        | 1108 ++++++++++++++++++++
 .../Impl/Transactions/TransactionsImpl.cs          |    8 +-
 .../Transactions/ITransactions.cs                  |    6 +
 .../java/org/apache/ignite/IgniteSpringBean.java   |    9 +
 .../zk/internal/ZookeeperDiscoveryImpl.java        |  131 ++-
 parent/pom.xml                                     |    4 +
 140 files changed, 11584 insertions(+), 1647 deletions(-)

diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandList.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
index 1b0595b..1e3df5d 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
@@ -71,7 +71,10 @@ public enum CommandList {
     METADATA("--meta", new MetadataCommand()),
 
     /** */
-    SHUTDOWN_POLICY("--shutdown-policy", new ShutdownPolicyCommand());
+    SHUTDOWN_POLICY("--shutdown-policy", new ShutdownPolicyCommand()),
+
+    /** */
+    TRACING_CONFIGURATION("--tracing-configuration", new TracingConfigurationCommand());
 
     /** Private values copy so there's no need in cloning it every time. */
     private static final CommandList[] VALUES = CommandList.values();
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/TracingConfigurationCommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/TracingConfigurationCommand.java
new file mode 100644
index 0000000..66eb2ba
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/TracingConfigurationCommand.java
@@ -0,0 +1,346 @@
+/*
+ * 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.commandline;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.logging.Logger;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientNode;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.tracing.configuration.TracingConfigurationArguments;
+import org.apache.ignite.internal.commandline.tracing.configuration.TracingConfigurationCommandArg;
+import org.apache.ignite.internal.commandline.tracing.configuration.TracingConfigurationSubcommand;
+import org.apache.ignite.internal.visor.tracing.configuration.VisorTracingConfigurationTask;
+import org.apache.ignite.internal.visor.tracing.configuration.VisorTracingConfigurationTaskArg;
+import org.apache.ignite.internal.visor.tracing.configuration.VisorTracingConfigurationTaskResult;
+import org.apache.ignite.spi.tracing.Scope;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
+import static org.apache.ignite.internal.commandline.CommandHandler.UTILITY_NAME;
+import static org.apache.ignite.internal.commandline.CommandList.TRACING_CONFIGURATION;
+import static org.apache.ignite.internal.commandline.CommandLogger.grouped;
+import static org.apache.ignite.internal.commandline.CommandLogger.join;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+import static org.apache.ignite.internal.commandline.tracing.configuration.TracingConfigurationSubcommand.GET_ALL;
+import static org.apache.ignite.internal.commandline.tracing.configuration.TracingConfigurationSubcommand.RESET_ALL;
+import static org.apache.ignite.internal.commandline.tracing.configuration.TracingConfigurationSubcommand.of;
+import static org.apache.ignite.spi.tracing.TracingConfigurationParameters.SAMPLING_RATE_ALWAYS;
+import static org.apache.ignite.spi.tracing.TracingConfigurationParameters.SAMPLING_RATE_NEVER;
+
+/**
+ * Commands associated with tracing configuration functionality.
+ */
+public class TracingConfigurationCommand implements Command<TracingConfigurationArguments> {
+    /** Arguments. */
+    private TracingConfigurationArguments args;
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(Logger log) {
+        if (!experimentalEnabled())
+            return;
+
+        Command.usage(
+            log,
+            "Print tracing configuration: ",
+            TRACING_CONFIGURATION);
+
+        Command.usage(
+            log,
+            "Print tracing configuration: ",
+            TRACING_CONFIGURATION,
+            TracingConfigurationSubcommand.GET_ALL.text(),
+            optional(TracingConfigurationCommandArg.SCOPE.argName(), join("|", Scope.values())));
+
+        Command.usage(
+            log,
+            "Print specific tracing configuration based on specified " +
+                TracingConfigurationCommandArg.SCOPE.argName() + " and " +
+                TracingConfigurationCommandArg.LABEL.argName() + ": ",
+            TRACING_CONFIGURATION,
+            TracingConfigurationSubcommand.GET.text(),
+            grouped(TracingConfigurationCommandArg.SCOPE.argName(), join("|", Scope.values())),
+            optional(TracingConfigurationCommandArg.LABEL.argName()));
+
+        Command.usage(
+            log,
+            "Reset all specific tracing configuration the to default. If " +
+                TracingConfigurationCommandArg.SCOPE.argName() +
+                " is specified, then remove all label specific configuration for the given scope and reset given scope" +
+                " specific configuration to the default, if " + TracingConfigurationCommandArg.SCOPE.argName() +
+                " is skipped then reset all tracing configurations to the default. Print tracing configuration.",
+            TRACING_CONFIGURATION,
+            RESET_ALL.text(),
+            optional(TracingConfigurationCommandArg.SCOPE.argName(), join("|", Scope.values())));
+
+        Command.usage(
+            log,
+            "Reset specific tracing configuration to the default. If both " +
+                TracingConfigurationCommandArg.SCOPE.argName() + " and " +
+                TracingConfigurationCommandArg.LABEL.argName() + " are specified then remove given configuration," +
+                " if only " + TracingConfigurationCommandArg.SCOPE.argName() +
+                " is specified then reset given configuration to the default." +
+                " Print reseted configuration.",
+            TRACING_CONFIGURATION,
+            TracingConfigurationSubcommand.RESET.text(),
+            grouped(TracingConfigurationCommandArg.SCOPE.argName(), join("|", Scope.values())),
+            optional(TracingConfigurationCommandArg.LABEL.argName()));
+
+        Command.usage(
+            log,
+            "Set new tracing configuration. If both " +
+                TracingConfigurationCommandArg.SCOPE.argName() + " and " +
+                TracingConfigurationCommandArg.LABEL.argName() + " are specified then add or override label" +
+                " specific configuration, if only " + TracingConfigurationCommandArg.SCOPE.argName() +
+                " is specified, then override scope specific configuration. Print applied configuration.",
+            TRACING_CONFIGURATION,
+            TracingConfigurationSubcommand.SET.text(),
+            grouped(TracingConfigurationCommandArg.SCOPE.argName(), join("|", Scope.values()),
+            optional(TracingConfigurationCommandArg.LABEL.argName()),
+            optional(TracingConfigurationCommandArg.SAMPLING_RATE.argName(),
+                "Decimal value between 0 and 1, " +
+                "where 0 means never and 1 means always. " +
+                "More or less reflects the probability of sampling specific trace."),
+            optional(TracingConfigurationCommandArg.INCLUDED_SCOPES.argName(),
+                "Set of scopes with comma as separator ",
+                join("|", Scope.values()))));
+    }
+
+    /**
+     * Execute tracing-configuration command.
+     *
+     * @param clientCfg Client configuration.
+     * @throws Exception If failed to execute tracing-configuration action.
+     */
+    @Override public Object execute(GridClientConfiguration clientCfg, Logger log) throws Exception {
+        if (experimentalEnabled()) {
+            try (GridClient client = Command.startClient(clientCfg)) {
+                UUID crdId = client.compute()
+                    //Only non client node can be coordinator.
+                    .nodes(node -> !node.isClient())
+                    .stream()
+                    .min(Comparator.comparingLong(GridClientNode::order))
+                    .map(GridClientNode::nodeId)
+                    .orElse(null);
+
+                VisorTracingConfigurationTaskResult res = executeTaskByNameOnNode(
+                    client,
+                    VisorTracingConfigurationTask.class.getName(),
+                    toVisorArguments(args),
+                    crdId,
+                    clientCfg
+                );
+
+                printResult(res, log::info);
+
+                return res;
+            }
+            catch (Throwable e) {
+                log.severe("Failed to execute tracing-configuration command='" + args.command().text() + "'");
+
+                throw e;
+            }
+        } else {
+            log.warning(String.format("For use experimental command add %s=true to JVM_OPTS in %s",
+                IGNITE_ENABLE_EXPERIMENTAL_COMMAND, UTILITY_NAME));
+
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        // If there is no arguments, use list command.
+        if (!argIter.hasNextSubArg()) {
+            args = new TracingConfigurationArguments.Builder(TracingConfigurationSubcommand.GET_ALL).build();
+
+            return;
+        }
+
+        TracingConfigurationSubcommand cmd = of(argIter.nextArg("Expected tracing configuration action."));
+
+        if (cmd == null)
+            throw new IllegalArgumentException("Expected correct tracing configuration action.");
+
+        TracingConfigurationArguments.Builder tracingConfigurationArgs = new TracingConfigurationArguments.Builder(cmd);
+
+        Scope scope = null;
+
+        String lb = null;
+
+        double samplingRate = SAMPLING_RATE_NEVER;
+
+        Set<Scope> includedScopes = new HashSet<>();
+
+        while (argIter.hasNextSubArg()) {
+            TracingConfigurationCommandArg arg =
+                CommandArgUtils.of(argIter.nextArg(""), TracingConfigurationCommandArg.class);
+
+            String strVal;
+
+            assert arg != null;
+
+            switch (arg) {
+                case SCOPE: {
+                    String peekedNextArg = argIter.peekNextArg();
+
+                    if (!TracingConfigurationCommandArg.args().contains(peekedNextArg)) {
+                        strVal = argIter.nextArg(
+                            "The scope should be specified. The following " +
+                                "values can be used: " + Arrays.toString(Scope.values()) + '.');
+
+                        try {
+                            scope = Scope.valueOf(strVal);
+                        }
+                        catch (IllegalArgumentException e) {
+                            throw new IllegalArgumentException(
+                                "Invalid scope '" + strVal + "'. The following " +
+                                    "values can be used: " + Arrays.toString(Scope.values()) + '.');
+                        }
+                    }
+
+                    break;
+                }
+                case LABEL: {
+                    lb = argIter.nextArg(
+                        "The label should be specified.");
+
+                    break;
+                }
+                case SAMPLING_RATE: {
+                    strVal = argIter.nextArg(
+                        "The sampling rate should be specified. Decimal value between 0 and 1 should be used.");
+
+                    try {
+                        samplingRate = Double.parseDouble(strVal);
+                    }
+                    catch (IllegalArgumentException e) {
+                        throw new IllegalArgumentException(
+                            "Invalid sampling-rate '" + strVal + "'. Decimal value between 0 and 1 should be used.");
+                    }
+
+                    if (samplingRate < SAMPLING_RATE_NEVER || samplingRate > SAMPLING_RATE_ALWAYS)
+                        throw new IllegalArgumentException(
+                            "Invalid sampling-rate '" + strVal + "'. Decimal value between 0 and 1 should be used.");
+
+                    break;
+                }
+                case INCLUDED_SCOPES: {
+                    Set<String> setStrVals = argIter.nextStringSet(
+                        "At least one supported scope should be specified.");
+
+                    for (String scopeStrVal : setStrVals) {
+                        try {
+                            includedScopes.add(Scope.valueOf(scopeStrVal));
+                        }
+                        catch (IllegalArgumentException e) {
+                            throw new IllegalArgumentException(
+                                "Invalid supported scope '" + scopeStrVal + "'. The following " +
+                                    "values can be used: " + Arrays.toString(Scope.values()) + '.');
+                        }
+                    }
+
+                    break;
+                }
+            }
+        }
+
+        // Scope is a mandatory attribute for all sub-commands except get_all and reset_all.
+        if ((cmd != GET_ALL && cmd != RESET_ALL) && scope == null) {
+            throw new IllegalArgumentException(
+                "Scope attribute is missing. Following values can be used: " + Arrays.toString(Scope.values()) + '.');
+        }
+
+        switch (cmd) {
+            case GET_ALL:
+            case RESET_ALL: {
+                tracingConfigurationArgs.withScope(scope);
+
+                break;
+            }
+
+            case RESET:
+            case GET: {
+                tracingConfigurationArgs.withScope(scope).withLabel(lb);
+
+                break;
+            }
+
+            case SET: {
+                tracingConfigurationArgs.withScope(scope).withLabel(lb).withSamplingRate(samplingRate).
+                    withIncludedScopes(includedScopes);
+
+                break;
+            }
+
+            default: {
+                // We should never get here.
+                assert false : "Unexpected tracing configuration argument [arg= " + cmd + ']';
+            }
+        }
+
+        args = tracingConfigurationArgs.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public TracingConfigurationArguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return TRACING_CONFIGURATION.toCommandName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean experimental() {
+        return true;
+    }
+
+    /**
+     * Print result.
+     *
+     * @param res Visor tracing configuration result.
+     * @param printer Printer.
+     */
+    private void printResult(VisorTracingConfigurationTaskResult res, Consumer<String> printer) {
+        res.print(printer);
+    }
+
+    /**
+     * Prepare task argument.
+     *
+     * @param args Argument from command line.
+     * @return Task argument.
+     */
+    private VisorTracingConfigurationTaskArg toVisorArguments(TracingConfigurationArguments args) {
+        return new VisorTracingConfigurationTaskArg(
+            args.command().visorBaselineOperation(),
+            args.scope(),
+            args.label(),
+            args.samplingRate(),
+            args.includedScopes()
+        );
+    }
+}
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationArguments.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationArguments.java
new file mode 100644
index 0000000..655f145
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationArguments.java
@@ -0,0 +1,180 @@
+/*
+ * 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.commandline.tracing.configuration;
+
+import java.util.Collections;
+import java.util.Set;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.visor.tracing.configuration.VisorTracingConfigurationItem;
+import org.apache.ignite.spi.tracing.Scope;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class contains all possible arguments after parsing tracing-configuration command input.
+ */
+ public final class TracingConfigurationArguments extends VisorTracingConfigurationItem {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Command. */
+    private TracingConfigurationSubcommand cmd;
+
+    /**
+     * Default constructor.
+     */
+    public TracingConfigurationArguments() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param cmd Command
+     * @param scope Specify the {@link Scope} of a trace's root span
+     *  to which some specific tracing configuration will be applied.
+     * @param lb Specifies the label of a traced operation. It's an optional attribute.
+     * @param samplingRate Number between 0 and 1 that more or less reflects the probability of sampling specific trace.
+     *  0 and 1 have special meaning here, 0 means never 1 means always. Default value is 0 (never).
+     * @param includedScopes Set of {@link Scope} that defines which sub-traces will be included in given trace.
+     *  In other words, if child's span scope is equals to parent's scope
+     *  or it belongs to the parent's span included scopes, then given child span will be attached to the current trace,
+     *  otherwise it'll be skipped.
+     *  See {@link Span#isChainable(Scope)} for more details.
+     */
+    private TracingConfigurationArguments(
+        TracingConfigurationSubcommand cmd,
+        Scope scope,
+        String lb,
+        double samplingRate,
+        Set<Scope> includedScopes
+    ) {
+        super(
+            scope,
+            lb,
+            samplingRate,
+            includedScopes);
+
+        this.cmd = cmd;
+    }
+
+    /**
+     * @return Command.
+     */
+    public TracingConfigurationSubcommand command() {
+        return cmd;
+    }
+
+    /**
+     * Builder of {@link TracingConfigurationArguments}.
+     */
+    @SuppressWarnings("PublicInnerClass") public static class Builder {
+        /** Counterpart of {@code TracingConfigurationArguments}'s command. */
+        private TracingConfigurationSubcommand cmd;
+
+        /** Counterpart of {@code TracingConfigurationArguments}'s scope. */
+        private Scope scope;
+
+        /** Counterpart of {@code TracingConfigurationArguments}'s label. */
+        private String lb;
+
+        /** Counterpart of {@code TracingConfigurationArguments}'s samplingRate. */
+        private double samplingRate;
+
+        /** Counterpart of {@code TracingConfigurationArguments}'s includedScopes. */
+        private Set<Scope> includedScopes;
+
+        /**
+         * Constructor.
+         *
+         * @param cmd {@code TracingConfigurationSubcommand} command.
+         */
+        public Builder(TracingConfigurationSubcommand cmd) {
+            this.cmd = cmd;
+        }
+
+        /**
+         * Builder method that allows to set scope.
+         * @param scope {@link Scope} of a trace's root span
+         *  to which some specific tracing configuration will be applied.
+         * @return Builder.
+         */
+        public @NotNull Builder withScope(Scope scope) {
+            this.scope = scope;
+
+            return this;
+        }
+
+        /**
+         * Builder method that allows to set sampling rate.
+         *
+         * @param samplingRate Number between 0 and 1 that more or less reflects the probability
+         *  of sampling specific trace.
+         *  0 and 1 have special meaning here, 0 means never 1 means always. Default value is 0 (never).
+         * @return Builder.
+         */
+        public @NotNull Builder withSamplingRate(double samplingRate) {
+            this.samplingRate = samplingRate;
+
+            return this;
+        }
+
+        /**
+         * Builder method that allows to set optional label attribute.
+         *
+         * @param lb Label of traced operation. It's an optional attribute.
+         * @return Builder
+         */
+        public @NotNull Builder withLabel(@Nullable String lb) {
+            this.lb = lb;
+
+            return this;
+        }
+
+        /**
+         * Builder method that allows to set included scopes.
+         *
+         * @param includedScopes Set of {@link Scope} that defines which sub-traces will be included in given trace.
+         * In other words, if child's span scope is equals to parent's scope
+         * or it belongs to the parent's span included scopes, then given child span will be attached to the current trace,
+         * otherwise it'll be skipped.
+         * See {@link Span#isChainable(Scope)} for more details.
+         * @return Builder
+         */
+        @SuppressWarnings("UnusedReturnValue")
+        public @NotNull Builder withIncludedScopes(Set<Scope> includedScopes) {
+            this.includedScopes = includedScopes == null ? Collections.emptySet() : includedScopes;
+
+            return this;
+        }
+
+        /**
+         * Builder's build method that produces {@link TracingConfigurationArguments}.
+         *
+         * @return {@code TracingConfigurationArguments} instance.
+         */
+        public TracingConfigurationArguments build() {
+            return new TracingConfigurationArguments(
+                cmd,
+                scope,
+                lb,
+                samplingRate,
+                includedScopes);
+        }
+    }
+}
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationCommandArg.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationCommandArg.java
new file mode 100644
index 0000000..3fd4fa4
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationCommandArg.java
@@ -0,0 +1,79 @@
+/*
+ * 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.commandline.tracing.configuration;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.commandline.CommandList;
+import org.apache.ignite.internal.commandline.argument.CommandArg;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.spi.tracing.Scope;
+
+/**
+ * {@link CommandList#TRACING_CONFIGURATION} command arguments.
+ */
+public enum TracingConfigurationCommandArg implements CommandArg {
+    /**
+     * Specify the {@link Scope} of a trace's root span to which some specific tracing configuration will be applied.
+     */
+    SCOPE("--scope"),
+
+    /** Specify the label of a traced operation. It's an optional attribute. */
+    LABEL("--label"),
+
+    /**
+     * Number between 0 and 1 that more or less reflects the probability of sampling specific trace. 0 and 1 have
+     * special meaning here, 0 means never 1 means always. Default value is 0 (never).
+     */
+    SAMPLING_RATE("--sampling-rate"),
+
+    /**
+     * Set of {@link Scope} that defines which sub-traces will be included in given trace. In other words, if child's
+     * span scope is equals to parent's scope or it belongs to the parent's span included scopes, then given child span
+     * will be attached to the current trace, otherwise it'll be skipped. See {@link
+     * Span#isChainable(Scope)} for more details.
+     */
+    INCLUDED_SCOPES("--included-scopes");
+
+    /** Arg name. */
+    private final String name;
+
+    /**
+     * Creates a new instance of tracing configuration argument.
+     *
+     * @param name command name.
+     */
+    TracingConfigurationCommandArg(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return List of arguments.
+     */
+    public static Set<String> args() {
+        return Arrays.stream(TracingConfigurationCommandArg.values())
+            .map(TracingConfigurationCommandArg::argName)
+            .collect(Collectors.toSet());
+    }
+
+    /** {inheritDoc} */
+    @Override public String argName() {
+        return name;
+    }
+}
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationSubcommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationSubcommand.java
new file mode 100644
index 0000000..a3b04c6
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/tracing/configuration/TracingConfigurationSubcommand.java
@@ -0,0 +1,104 @@
+/*
+ * 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.commandline.tracing.configuration;
+
+import org.apache.ignite.internal.visor.tracing.configuration.VisorTracingConfigurationOperation;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Set of tracing configuration commands.
+ */
+public enum TracingConfigurationSubcommand {
+    /** Get specific tracing configuration. */
+    GET("get", VisorTracingConfigurationOperation.GET),
+
+    /** Get tracing configuration. */
+    GET_ALL("get_all", VisorTracingConfigurationOperation.GET_ALL),
+
+    /** Reset specific tracing configuration to default. */
+    RESET("reset", VisorTracingConfigurationOperation.RESET),
+
+    /**
+     * Reset all scope specific tracing configuration to default,
+     * or reset all tracing configurations to default if scope is not specified.
+     */
+    RESET_ALL("reset_all", VisorTracingConfigurationOperation.RESET_ALL),
+
+    /** Set new tracing configuration. */
+    SET("set", VisorTracingConfigurationOperation.SET);
+
+    /** Enumerated values. */
+    private static final TracingConfigurationSubcommand[] VALS = values();
+
+    /** Name. */
+    private final String name;
+
+    /** Corresponding visor tracing configuration operation. */
+    private final VisorTracingConfigurationOperation visorOperation;
+
+    /**
+     * @param name Name.
+     * @param visorOperation Corresponding visor tracing configuration operation.
+     */
+    TracingConfigurationSubcommand(String name, VisorTracingConfigurationOperation visorOperation) {
+        this.name = name;
+        this.visorOperation = visorOperation;
+    }
+
+    /**
+     * @param text Command text.
+     * @return Command for the text.
+     */
+    public static TracingConfigurationSubcommand of(String text) {
+        for (TracingConfigurationSubcommand cmd : TracingConfigurationSubcommand.values()) {
+            if (cmd.text().equalsIgnoreCase(text))
+                return cmd;
+        }
+
+        return null;
+    }
+
+    /**
+     * @return Name.
+     */
+    public String text() {
+        return name;
+    }
+
+    /**
+     * @return {@link VisorTracingConfigurationOperation} which is associated with tracing configuration subcommand.
+     */
+    public VisorTracingConfigurationOperation visorBaselineOperation() {
+        return visorOperation;
+    }
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static TracingConfigurationSubcommand fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return name;
+    }
+}
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
index b8a2534..7eed217 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.commandline;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.UUID;
@@ -37,6 +38,7 @@ import org.apache.ignite.internal.visor.tx.VisorTxOperation;
 import org.apache.ignite.internal.visor.tx.VisorTxProjection;
 import org.apache.ignite.internal.visor.tx.VisorTxSortOrder;
 import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
+import org.apache.ignite.spi.tracing.Scope;
 import org.apache.ignite.testframework.junits.SystemPropertiesRule;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.jetbrains.annotations.Nullable;
@@ -595,6 +597,195 @@ public class CommandHandlerParsingTest {
     }
 
     /**
+     * Negative argument validation test for tracing-configuration command.
+     *
+     * validate that following tracing-configuration arguments validated as expected:
+     * <ul>
+     *     <li>
+     *         reset_all, get_all
+     *         <ul>
+     *             <li>
+     *                 --scope
+     *                 <ul>
+     *                     <li>
+     *                         if value is missing:
+     *                          IllegalArgumentException (The scope should be specified. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX].")
+     *                     </li>
+     *                     <li>
+     *                         if unsupported value is used:
+     *                          IllegalArgumentException (Invalid scope 'aaa'. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX])
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *         </ul>
+     *     </li>
+     *     <li>
+     *         reset, get:
+     *         <ul>
+     *             <li>
+     *                 --scope
+     *                 <ul>
+     *                     <li>
+     *                         if value is missing:
+     *                          IllegalArgumentException (The scope should be specified. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX].")
+     *                     </li>
+     *                     <li>
+     *                         if unsupported value is used:
+     *                          IllegalArgumentException (Invalid scope 'aaa'. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX])
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *             <li>
+     *                 --label
+     *                 <ul>
+     *                     <li>
+     *                         if value is missing:
+     *                          IllegalArgumentException (The label should be specified.)
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *         </ul>
+     *     </li>
+     *     <li>
+     *         set:
+     *         <ul>
+     *             <li>
+     *                 --scope
+     *                 <ul>
+     *                     <li>
+     *                         if value is missing:
+     *                          IllegalArgumentException (The scope should be specified. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX].")
+     *                     </li>
+     *                     <li>
+     *                         if unsupported value is used:
+     *                          IllegalArgumentException (Invalid scope 'aaa'. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX])
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *             <li>
+     *                 --label
+     *                 <ul>
+     *                     <li>
+     *                          if value is missing:
+     *                              IllegalArgumentException (The label should be specified.)
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *             <li>
+     *                 --sampling-rate
+     *                 <ul>
+     *                     <li>
+     *                          if value is missing:
+     *                              IllegalArgumentException (The sampling-rate should be specified. Decimal value between 0 and 1 should be used.)
+     *                     </li>
+     *                     <li>
+     *                          if unsupported value is used:
+     *                              IllegalArgumentException (Invalid samling-rate 'aaa'. Decimal value between 0 and 1 should be used.)
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *             <li>
+     *                 --included-scopes
+     *                 <ul>
+     *                     <li>
+     *                          if value is missing:
+     *                              IllegalArgumentException (At least one supported scope should be specified.)
+     *                     </li>
+     *                     <li>
+     *                          if unsupported value is used:
+     *                              IllegalArgumentException (Invalid supported scope: aaa. The following values can be used: [DISCOVERY, EXCHANGE, COMMUNICATION, TX].)
+     *                     </li>
+     *                 </ul>
+     *             </li>
+     *         </ul>
+     *     </li>
+     * </ul>
+     */
+    @Test
+    public void testTracingConfigurationArgumentsValidation() {
+        // reset
+        assertParseArgsThrows("The scope should be specified. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "reset", "--scope");
+
+        assertParseArgsThrows("Invalid scope 'aaa'. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "reset", "--scope", "aaa");
+
+        assertParseArgsThrows("The label should be specified.",
+            "--tracing-configuration", "reset", "--label");
+
+        // reset all
+        assertParseArgsThrows("The scope should be specified. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "reset_all", "--scope");
+
+        assertParseArgsThrows("Invalid scope 'aaa'. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "reset_all", "--scope", "aaa");
+
+        // get
+        assertParseArgsThrows("The scope should be specified. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "get", "--scope");
+
+        assertParseArgsThrows("Invalid scope 'aaa'. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "get", "--scope", "aaa");
+
+        assertParseArgsThrows("The label should be specified.",
+            "--tracing-configuration", "get", "--label");
+
+        // get all
+        assertParseArgsThrows("The scope should be specified. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "get_all", "--scope");
+
+        assertParseArgsThrows("Invalid scope 'aaa'. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "get_all", "--scope", "aaa");
+
+        // set
+        assertParseArgsThrows("The scope should be specified. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "set", "--scope");
+
+        assertParseArgsThrows("Invalid scope 'aaa'. The following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "set", "--scope", "aaa");
+
+        assertParseArgsThrows("The label should be specified.",
+            "--tracing-configuration", "set", "--label");
+
+        assertParseArgsThrows("The sampling rate should be specified. Decimal value between 0 and 1 should be used.",
+            "--tracing-configuration", "set", "--sampling-rate");
+
+        assertParseArgsThrows("Invalid sampling-rate 'aaa'. Decimal value between 0 and 1 should be used.",
+            "--tracing-configuration", "set", "--sampling-rate", "aaa");
+
+        assertParseArgsThrows("Invalid sampling-rate '-1'. Decimal value between 0 and 1 should be used.",
+            "--tracing-configuration", "set", "--sampling-rate", "-1");
+
+        assertParseArgsThrows("Invalid sampling-rate '2'. Decimal value between 0 and 1 should be used.",
+            "--tracing-configuration", "set", "--sampling-rate", "2");
+
+        assertParseArgsThrows("At least one supported scope should be specified.",
+            "--tracing-configuration", "set", "--included-scopes");
+
+        assertParseArgsThrows("Invalid supported scope 'aaa'. The following values can be used: "
+                + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "set", "--included-scopes", "TX,aaa");
+    }
+
+    /**
+     * Positive argument validation test for tracing-configuration command.
+     */
+    @Test
+    public void testTracingConfigurationArgumentsValidationMandatoryArgumentSet() {
+        parseArgs(asList("--tracing-configuration"));
+
+        parseArgs(asList("--tracing-configuration", "get_all"));
+
+        assertParseArgsThrows("Scope attribute is missing. Following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "reset");
+
+        assertParseArgsThrows("Scope attribute is missing. Following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "get");
+
+        assertParseArgsThrows("Scope attribute is missing. Following values can be used: "
+            + Arrays.toString(Scope.values()) + '.', "--tracing-configuration", "set");
+    }
+
+    /**
      * Test checks that option {@link CommonArgParser#CMD_VERBOSE} is parsed
      * correctly and if it is not present, it takes the default value
      * {@code false}.
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java b/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java
index cc748e5..861ce96 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java
@@ -33,6 +33,7 @@ import org.apache.ignite.util.GridCommandHandlerIndexingWithSSLTest;
 import org.apache.ignite.util.GridCommandHandlerMetadataTest;
 import org.apache.ignite.util.GridCommandHandlerSslTest;
 import org.apache.ignite.util.GridCommandHandlerTest;
+import org.apache.ignite.util.GridCommandHandlerTracingConfigurationTest;
 import org.apache.ignite.util.GridCommandHandlerWithSSLTest;
 import org.apache.ignite.util.KillCommandsCommandShTest;
 import org.junit.runner.RunWith;
@@ -66,6 +67,8 @@ import org.junit.runners.Suite;
 
     BaselineEventsLocalTest.class,
     BaselineEventsRemoteTest.class,
+
+    GridCommandHandlerTracingConfigurationTest.class,
 })
 public class IgniteControlUtilityTestSuite {
 }
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
index 09722a1..b3805f9 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
@@ -24,6 +24,7 @@ import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumMap;
 import java.util.HashSet;
 import java.util.List;
@@ -99,6 +100,7 @@ import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_UN
 import static org.apache.ignite.internal.commandline.CommandHandler.UTILITY_NAME;
 import static org.apache.ignite.internal.commandline.CommandList.BASELINE;
 import static org.apache.ignite.internal.commandline.CommandList.METADATA;
+import static org.apache.ignite.internal.commandline.CommandList.TRACING_CONFIGURATION;
 import static org.apache.ignite.internal.commandline.CommandList.WAL;
 import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_VERBOSE;
 import static org.apache.ignite.internal.commandline.OutputFormat.MULTI_LINE;
@@ -1594,6 +1596,7 @@ public class GridCommandHandlerClusterByClassTest extends GridCommandHandlerClus
 
         cmdArgs.put(WAL, asList("print", "delete"));
         cmdArgs.put(METADATA, asList("help", "list"));
+        cmdArgs.put(TRACING_CONFIGURATION, Collections.singletonList("get_all"));
 
         String warning = String.format(
             "For use experimental command add %s=true to JVM_OPTS in %s",
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTracingConfigurationTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTracingConfigurationTest.java
new file mode 100644
index 0000000..eb7abe2
--- /dev/null
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTracingConfigurationTest.java
@@ -0,0 +1,390 @@
+/*
+ * 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.util;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.commandline.CommandHandler;
+import org.apache.ignite.internal.commandline.TracingConfigurationCommand;
+import org.apache.ignite.internal.visor.tracing.configuration.VisorTracingConfigurationTaskResult;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.TracingConfigurationCoordinates;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.apache.ignite.spi.tracing.TracingConfigurationParameters;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK;
+import static org.apache.ignite.spi.tracing.Scope.COMMUNICATION;
+import static org.apache.ignite.spi.tracing.Scope.EXCHANGE;
+import static org.apache.ignite.spi.tracing.Scope.TX;
+
+/**
+ * Tests for {@link TracingConfigurationCommand}
+ */
+public class GridCommandHandlerTracingConfigurationTest extends GridCommandHandlerClusterPerMethodAbstractTest {
+    /** Default configuration map. */
+    private static final Map<TracingConfigurationCoordinates, TracingConfigurationParameters> DFLT_CONFIG_MAP =
+        new HashMap<>();
+
+    /** TX scope specific coordinates to be used within several tests. */
+    private static final TracingConfigurationCoordinates TX_SCOPE_SPECIFIC_COORDINATES =
+        new TracingConfigurationCoordinates.Builder(TX).build();
+
+    /** EXCHANGE scope specific coordinates to be used within several tests. */
+    private static final TracingConfigurationCoordinates EXCHANGE_SCOPE_SPECIFIC_COORDINATES =
+        new TracingConfigurationCoordinates.Builder(EXCHANGE).build();
+
+    /** Updated scope specific parameters to be used within several tests. */
+    private static final TracingConfigurationParameters SOME_SCOPE_SPECIFIC_PARAMETERS =
+        new TracingConfigurationParameters.Builder().withSamplingRate(0.75).
+            withIncludedScopes(Collections.singleton(COMMUNICATION)).build();
+
+    /** TX Label specific coordinates to be used within several tests. */
+    private static final TracingConfigurationCoordinates TX_LABEL_SPECIFIC_COORDINATES =
+        new TracingConfigurationCoordinates.Builder(TX).withLabel("label").build();
+
+    /** Updated label specific parameters to be used within several tests. */
+    private static final TracingConfigurationParameters SOME_LABEL_SPECIFIC_PARAMETERS =
+        new TracingConfigurationParameters.Builder().withSamplingRate(0.111).
+            withIncludedScopes(Collections.singleton(EXCHANGE)).build();
+
+    static {
+        DFLT_CONFIG_MAP.put(
+            new TracingConfigurationCoordinates.Builder(Scope.TX).build(),
+            TracingConfigurationManager.DEFAULT_TX_CONFIGURATION);
+
+        DFLT_CONFIG_MAP.put(
+            new TracingConfigurationCoordinates.Builder(Scope.COMMUNICATION).build(),
+            TracingConfigurationManager.DEFAULT_COMMUNICATION_CONFIGURATION);
+
+        DFLT_CONFIG_MAP.put(
+            new TracingConfigurationCoordinates.Builder(Scope.EXCHANGE).build(),
+            TracingConfigurationManager.DEFAULT_EXCHANGE_CONFIGURATION);
+
+        DFLT_CONFIG_MAP.put(
+            new TracingConfigurationCoordinates.Builder(Scope.DISCOVERY).build(),
+            TracingConfigurationManager.DEFAULT_DISCOVERY_CONFIGURATION);
+    }
+
+    /** */
+    protected IgniteEx ignite;
+
+    /** */
+    private static CommandHandler hnd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        ignite = startGrids(2);
+
+        hnd = new CommandHandler();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        // Cleanup configuration.
+        grid(0).tracingConfiguration().resetAll(null);
+
+        // Populate tracing with some custom configurations.
+        grid(0).tracingConfiguration().set(
+            TX_SCOPE_SPECIFIC_COORDINATES,
+            SOME_SCOPE_SPECIFIC_PARAMETERS);
+
+        grid(0).tracingConfiguration().set(
+            TX_LABEL_SPECIFIC_COORDINATES,
+            SOME_LABEL_SPECIFIC_PARAMETERS);
+
+        grid(0).tracingConfiguration().set(
+            EXCHANGE_SCOPE_SPECIFIC_COORDINATES,
+            SOME_SCOPE_SPECIFIC_PARAMETERS);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        // Do nothing;
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration" without arguments
+     * tracing configuration for all scopes will be returned.
+     */
+    @Test
+    public void testTracingConfigurationWithoutSubCommandsReturnsTracingConfiguratoinForAllScopes() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration"));
+
+        Map<TracingConfigurationCoordinates, TracingConfigurationParameters> expTracingCfg =
+            new HashMap<>(DFLT_CONFIG_MAP);
+
+        expTracingCfg.put(TX_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+        expTracingCfg.put(TX_LABEL_SPECIFIC_COORDINATES, SOME_LABEL_SPECIFIC_PARAMETERS);
+        expTracingCfg.put(EXCHANGE_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expTracingCfg.forEach(expRes::add);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration get_all --scope TX"
+     * TX based tracing configuration will be returned: both scope specific and label specific.
+     */
+    @Test
+    public void testGetAllWithScopeReturnsOnlySpecifiedScopeSpecificConfiguratoin() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "get_all", "--scope", "TX"));
+
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(TX_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+        expRes.add(TX_LABEL_SPECIFIC_COORDINATES, SOME_LABEL_SPECIFIC_PARAMETERS);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration get_all" without scope
+     * tracing configuration for all scopes will be returned.
+     */
+    @Test
+    public void testGetAllWithoutScopeReturnsTracingConfigurationsForAllScopes() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "get_all"));
+
+        // Check command result.
+        Map<TracingConfigurationCoordinates, TracingConfigurationParameters> expTracingCfg =
+            new HashMap<>(DFLT_CONFIG_MAP);
+
+        expTracingCfg.put(TX_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+        expTracingCfg.put(TX_LABEL_SPECIFIC_COORDINATES, SOME_LABEL_SPECIFIC_PARAMETERS);
+        expTracingCfg.put(EXCHANGE_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expTracingCfg.forEach(expRes::add);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration get --scope TX"
+     * TX specific and only TX specific tracing configuration will be returned:
+     * TX-label specific configuration not expected.
+     */
+    @Test
+    public void testGetWithScopeReturnsScopeSpecificConfiguratoin() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "get", "--scope", "TX"));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(TX_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration get --scope TX --label label"
+     * TX label specific and only TX label specific tracing configuration will be returned:
+     * TX specific configuration not expected.
+     */
+    @Test
+    public void testGetWithScopeAndLabelReturnsLabelSpecificConfigurationIfSuchOneExists() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "get", "--scope",
+            "TX", "--label", "label"));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(TX_LABEL_SPECIFIC_COORDINATES, SOME_LABEL_SPECIFIC_PARAMETERS);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration reset_all --scope TX"
+     * TX based configuration will be reseted and returned:
+     */
+    @Test
+    public void testResetAllWithScopeResetsScopeBasedConfigurationAndReturnsIt() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "reset_all", "--scope", "TX"));
+
+        // Ensure that configuration was actually reseted.
+        assertEquals(
+            Collections.singletonMap(
+                TX_SCOPE_SPECIFIC_COORDINATES,
+                TracingConfigurationManager.DEFAULT_TX_CONFIGURATION),
+            grid(0).tracingConfiguration().getAll(TX));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(TX_SCOPE_SPECIFIC_COORDINATES, TracingConfigurationManager.DEFAULT_TX_CONFIGURATION);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration reset_all"
+     * Whole tracing configurations will be reseted and returned.
+     */
+    @Test
+    public void testResetAllWithoutScopeResetsTracingConfigurationForAllScopesAndReturnsIt() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "reset_all"));
+
+        // Ensure that configuration was actually reseted.
+        assertEquals(
+            Collections.singletonMap(
+                TX_SCOPE_SPECIFIC_COORDINATES,
+                TracingConfigurationManager.DEFAULT_TX_CONFIGURATION),
+            grid(0).tracingConfiguration().getAll(TX));
+
+        assertEquals(
+            Collections.singletonMap(
+                EXCHANGE_SCOPE_SPECIFIC_COORDINATES,
+                TracingConfigurationManager.DEFAULT_EXCHANGE_CONFIGURATION),
+            grid(0).tracingConfiguration().getAll(EXCHANGE));
+
+        // Check command result.
+        Map<TracingConfigurationCoordinates, TracingConfigurationParameters> expTracingCfg =
+            new HashMap<>(DFLT_CONFIG_MAP);
+
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expTracingCfg.forEach(expRes::add);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration reset --scope TX"
+     * TX scope specific configuration will be reseted, TX label specific configuration should stay unchanged.
+     * Whole TX based configuration should be returned.
+     */
+    @Test
+    public void testResetWithScopeResetsScopeSpecificConfiguratoinAndReturnesScopeBasedConfiguration() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "reset", "--scope", "TX"));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(TX_SCOPE_SPECIFIC_COORDINATES, TracingConfigurationManager.DEFAULT_EXCHANGE_CONFIGURATION);
+        expRes.add(TX_LABEL_SPECIFIC_COORDINATES, SOME_LABEL_SPECIFIC_PARAMETERS);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration reset --scope TX --label label"
+     * TX label specific configuration will be removed, TX scope specific configuration should stay unchanged.
+     * Whole TX based configuration should be returned.
+     */
+    @Test
+    public void testResetWithScopeAndLabelResetsLabelSpecificConfiguratoinAndReturnesScopeBasedConfiguration() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "reset", "--scope", "TX",
+            "--label", "label"));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(TX_SCOPE_SPECIFIC_COORDINATES, SOME_SCOPE_SPECIFIC_PARAMETERS);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration set --scope TX --sampling-rate 0.123
+     * --included-scopes COMMUNICATION,EXCHANGE"
+     * TX scope specific configuration should be updated.
+     * Whole TX based configuration should be returned.
+     */
+    @Test
+    public void testSetWithScopeSetsScopeSpecificConfiguratoinAndReturnesScopeBasedConfiguration() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "set", "--scope", "TX",
+            "--sampling-rate", "0.123", "--included-scopes", "COMMUNICATION,EXCHANGE"));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(
+            TX_SCOPE_SPECIFIC_COORDINATES,
+            new TracingConfigurationParameters.Builder().
+                withSamplingRate(0.123).
+                withIncludedScopes(new HashSet<>(Arrays.asList(COMMUNICATION, EXCHANGE))).build());
+
+        expRes.add(TX_LABEL_SPECIFIC_COORDINATES, SOME_LABEL_SPECIFIC_PARAMETERS);
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Ensure that in case of "--tracing-configuration set --scope TX --label label --sampling-rate 0.123
+     * --included-scopes COMMUNICATION,EXCHANGE"
+     * TX label specific configuration should be updated.
+     * Whole TX based configuration should be returned.
+     */
+    @Test
+    public void testSetWithScopeAndLabelSetsLabelSpecificConfiguratoinAndReturnsScopeBasedConfiguration() {
+        assertEquals(EXIT_CODE_OK, execute(hnd, "--tracing-configuration", "set", "--scope", "TX",
+            "--label", "label", "--sampling-rate", "0.123", "--included-scopes", "COMMUNICATION,EXCHANGE"));
+
+        // Check command result.
+        VisorTracingConfigurationTaskResult expRes = new VisorTracingConfigurationTaskResult();
+
+        expRes.add(
+            TX_SCOPE_SPECIFIC_COORDINATES,
+            SOME_SCOPE_SPECIFIC_PARAMETERS);
+
+        expRes.add(
+            TX_LABEL_SPECIFIC_COORDINATES,
+            new TracingConfigurationParameters.Builder().
+                withSamplingRate(0.123).
+                withIncludedScopes(new HashSet<>(Arrays.asList(COMMUNICATION, EXCHANGE))).build());
+
+        verifyResult(expRes);
+    }
+
+    /**
+     * Verify that expected result equals got one.
+     *
+     * @param expRes Expected command result.
+     */
+    private void verifyResult(VisorTracingConfigurationTaskResult expRes) {
+        VisorTracingConfigurationTaskResult gotRes = hnd.getLastOperationResult();
+
+        assertNotNull(gotRes);
+
+        assertNotNull(gotRes.tracingConfigurations());
+
+        assertTrue(expRes.tracingConfigurations().containsAll(gotRes.tracingConfigurations()) &&
+            gotRes.tracingConfigurations().containsAll(expRes.tracingConfigurations()));
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 7332eaa..98f9937 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -32,9 +32,12 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.lang.IgniteExperimental;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.plugin.IgnitePlugin;
 import org.apache.ignite.plugin.PluginNotFoundException;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -741,4 +744,18 @@ public interface Ignite extends AutoCloseable {
      * @return Snapshot manager.
      */
     public IgniteSnapshot snapshot();
+
+    /**
+     * Returns the {@link TracingConfigurationManager} instance that allows to
+     * <ul>
+     *     <li>Configure tracing parameters such as sampling rate for the specific tracing coordinates
+     *          such as scope and label.</li>
+     *     <li>Retrieve the most specific tracing parameters for the specified tracing coordinates (scope and label)</li>
+     *     <li>Restore the tracing parameters for the specified tracing coordinates to the default.</li>
+     *     <li>List all pairs of tracing configuration coordinates and tracing configuration parameters.</li>
+     * </ul>
+     * @return {@link TracingConfigurationManager} instance.
+     */
+    @IgniteExperimental
+    public @NotNull TracingConfigurationManager tracingConfiguration();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
index 2bb7101..852f923 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
@@ -124,4 +124,11 @@ public interface IgniteTransactions {
      * @throws NullPointerException if label is null.
      */
     public IgniteTransactions withLabel(String lb);
+
+    /**
+     * Returns an instance of {@code IgniteTransactions} tran will trace every transaction.
+     *
+     * @return Trace-enabled transactions intance.
+     */
+    public IgniteTransactions withTracing();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index cc0fe54..b5bbcb3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -84,6 +84,7 @@ import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi;
 import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi;
 import org.apache.ignite.spi.metric.MetricExporterSpi;
 import org.apache.ignite.spi.systemview.SystemViewExporterSpi;
+import org.apache.ignite.spi.tracing.TracingSpi;
 import org.apache.ignite.ssl.SslContextFactory;
 import org.jetbrains.annotations.Nullable;
 
@@ -435,6 +436,9 @@ public class IgniteConfiguration {
     /** System view exporter SPI. */
     private SystemViewExporterSpi[] sysViewExporterSpi;
 
+    /** Tracing SPI. */
+    private TracingSpi tracingSpi;
+
     /** Cache configurations. */
     private CacheConfiguration[] cacheCfg;
 
@@ -644,6 +648,7 @@ public class IgniteConfiguration {
         encryptionSpi = cfg.getEncryptionSpi();
         metricExporterSpi = cfg.getMetricExporterSpi();
         sysViewExporterSpi = cfg.getSystemViewExporterSpi();
+        tracingSpi = cfg.getTracingSpi();
 
         commFailureRslvr = cfg.getCommunicationFailureResolver();
 
@@ -2523,6 +2528,27 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Set fully configured instance of {@link TracingSpi}.
+     *
+     * @param tracingSpi Fully configured instance of {@link TracingSpi}.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setTracingSpi(TracingSpi tracingSpi) {
+        this.tracingSpi = tracingSpi;
+
+        return this;
+    }
+
+    /**
+     * Gets fully configured tracing SPI implementation.
+     *
+     * @return Tracing SPI implementation.
+     */
+    public TracingSpi getTracingSpi() {
+        return tracingSpi;
+    }
+
+    /**
      * Gets address resolver for addresses mapping determination.
      *
      * @return Address resolver.
diff --git a/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java b/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java
index 09f23bc..16d030c 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/DiscoveryEvent.java
@@ -19,9 +19,11 @@ package org.apache.ignite.events;
 
 import java.util.Collection;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Grid discovery event.
@@ -75,6 +77,12 @@ public class DiscoveryEvent extends EventAdapter {
     /** Collection of nodes corresponding to topology version. */
     private Collection<ClusterNode> topSnapshot;
 
+    /** Template to generate {@link #message()} lazily. Will be joined with {@link #eventNode()} converted to string. */
+    private volatile String msgTemplate;
+
+    /** Span. */
+    private transient Span span;
+
     /** {@inheritDoc} */
     @Override public String shortDisplay() {
         return name() + ": id8=" + U.id8(evtNode.id()) + ", ip=" + F.first(evtNode.addresses());
@@ -154,6 +162,57 @@ public class DiscoveryEvent extends EventAdapter {
         this.topSnapshot = topSnapshot;
     }
 
+    /**
+     * Template to generate {@link #message()} lazily. Will be joined with {@link #eventNode()} converted to string.
+     *
+     * @param msgTemplate Template.
+     */
+    public void messageTemplate(String msgTemplate) {
+        this.msgTemplate = msgTemplate;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String message() {
+        String msg = super.message();
+
+        if (msg != null)
+            return msg;
+
+        if (msgTemplate == null)
+            return null;
+
+        synchronized (this) {
+            msg = super.message();
+
+            if (msg != null)
+                return msg;
+
+            msg = msgTemplate + eventNode();
+
+            message(msg);
+        }
+
+        return msg;
+    }
+
+    /**
+     * Gets span instance.
+     *
+     * @return Span.
+     */
+    public Span span() {
+        return span;
+    }
+
+    /**
+     * Set span.
+     *
+     * @param span Span.
+     */
+    public void span(Span span) {
+        this.span = span;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DiscoveryEvent.class, this,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index e0c9c88..c417e11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -76,6 +76,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.processors.tracing.Tracing;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -239,6 +240,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public DistributedConfigurationProcessor distributedConfiguration();
 
     /**
+     * Gets tracing processor.
+     *
+     * @return Tracing processor.
+     */
+    public Tracing tracing();
+
+    /**
      * Gets task session processor.
      *
      * @return Session processor.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 67375d8..e1a9327 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
+import org.apache.ignite.internal.managers.tracing.GridTracingManager;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
@@ -95,6 +96,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.processors.tracing.Tracing;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -173,6 +175,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     @GridToStringExclude
     private GridEncryptionManager encryptionMgr;
 
+    /** */
+    @GridToStringExclude
+    private GridTracingManager tracingMgr;
+
     /*
      * Processors.
      * ==========
@@ -630,6 +636,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             indexingMgr = (GridIndexingManager)comp;
         else if (comp instanceof GridEncryptionManager)
             encryptionMgr = (GridEncryptionManager)comp;
+        else if (comp instanceof GridTracingManager)
+            tracingMgr = (GridTracingManager) comp;
 
         /*
          * Processors.
@@ -836,6 +844,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public Tracing tracing() {
+        return tracingMgr;
+    }
+
+    /** {@inheritDoc} */
     @Override public GridTaskSessionProcessor session() {
         return sesProc;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
index 5e65876..6d40803 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
@@ -93,10 +93,18 @@ public enum IgniteComponentType {
         "ignite-schedule"
     ),
 
+    /** */
     COMPRESSION(
         CompressionProcessor.class.getName(),
         "org.apache.ignite.internal.processors.compress.CompressionProcessorImpl",
         "ignite-compress"
+    ),
+
+    /** OpenCensus tracing implementation. */
+    TRACING(
+        null,
+        "org.apache.ignite.spi.tracing.opencensus.OpenCensusTracingSpi",
+        "ignite-opencensus"
     );
 
     /** No-op class name. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
index 0dccb8a..ba5427b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
@@ -98,6 +98,9 @@ public enum IgniteFeatures {
     /** Persistence caches can be snapshot.  */
     PERSISTENCE_CACHE_SNAPSHOT(23),
 
+    /** Tracing. */
+    TRACING(26),
+
     /** Distributed change timeout for dump long operations. */
     DISTRIBUTED_CHANGE_LONG_OPERATIONS_DUMP_TIMEOUT(30),
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index f43283b..77ff68e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -126,6 +126,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
+import org.apache.ignite.internal.managers.tracing.GridTracingManager;
 import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.internal.processors.GridProcessor;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -231,7 +232,9 @@ import org.apache.ignite.spi.IgniteSpi;
 import org.apache.ignite.spi.IgniteSpiVersionCheckException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
 import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
@@ -1134,9 +1137,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             cfg.getMarshaller().setContext(ctx.marshallerContext());
 
-            GridInternalSubscriptionProcessor subscriptionProc = new GridInternalSubscriptionProcessor(ctx);
-
-            startProcessor(subscriptionProc);
+            startProcessor(new GridInternalSubscriptionProcessor(ctx));
 
             ClusterProcessor clusterProc = new ClusterProcessor(ctx);
 
@@ -1195,6 +1196,12 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             // Start SPI managers.
             // NOTE: that order matters as there are dependencies between managers.
+            try {
+                startManager(new GridTracingManager(ctx, false));
+            }
+            catch (IgniteCheckedException e) {
+                startManager(new GridTracingManager(ctx, true));
+            }
             startManager(new GridMetricManager(ctx));
             startManager(new GridSystemViewManager(ctx));
             startManager(new GridIoManager(ctx));
@@ -1931,6 +1938,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         addSpiAttributes(cfg.getCheckpointSpi());
         addSpiAttributes(cfg.getLoadBalancingSpi());
         addSpiAttributes(cfg.getDeploymentSpi());
+        addSpiAttributes(cfg.getTracingSpi());
 
         // Set user attributes for this node.
         if (cfg.getUserAttributes() != null) {
@@ -3998,6 +4006,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public @NotNull TracingConfigurationManager tracingConfiguration() {
+        guard();
+
+        try {
+            return ctx.tracing().configuration();
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteEncryption encryption() {
         return ctx.encryption();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 8481c10..7d64ddf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -135,6 +135,7 @@ import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi;
 import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi;
 import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.spi.systemview.jmx.JmxSystemViewExporterSpi;
+import org.apache.ignite.spi.tracing.NoopTracingSpi;
 import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.thread.IgniteThreadPoolExecutor;
@@ -2543,6 +2544,9 @@ public class IgnitionEx {
                 else
                     cfg.setSystemViewExporterSpi(new JmxSystemViewExporterSpi());
             }
+
+            if (cfg.getTracingSpi() == null)
+                cfg.setTracingSpi(new NoopTracingSpi());
         }
 
         /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 452a5c4..79c4d08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -105,11 +105,14 @@ import org.apache.ignite.internal.processors.platform.message.PlatformMessageFil
 import org.apache.ignite.internal.processors.pool.PoolProcessor;
 import org.apache.ignite.internal.processors.security.OperationSecurityContext;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanTags;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -164,6 +167,10 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYS
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.isReservedGridIoPolicy;
 import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+import static org.apache.ignite.internal.processors.tracing.MTC.support;
+import static org.apache.ignite.internal.processors.tracing.SpanType.COMMUNICATION_ORDERED_PROCESS;
+import static org.apache.ignite.internal.processors.tracing.SpanType.COMMUNICATION_REGULAR_PROCESS;
+import static org.apache.ignite.internal.processors.tracing.messages.TraceableMessagesTable.traceName;
 import static org.apache.ignite.internal.util.nio.GridNioBackPressureControl.threadProcessingMessage;
 import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.ATTR_PAIRED_CONN;
 import static org.apache.ignite.spi.communication.tcp.internal.TcpConnectionIndexAwareMessage.UNDEFINED_CONNECTION_INDEX;
@@ -1407,9 +1414,11 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         final byte plc,
         final IgniteRunnable msgC
     ) throws IgniteCheckedException {
-        Runnable c = new Runnable() {
-            @Override public void run() {
+        Runnable c = new TraceRunnable(ctx.tracing(), COMMUNICATION_REGULAR_PROCESS) {
+            @Override public void execute() {
                 try {
+                    MTC.span().addTag(SpanTags.MESSAGE, () -> traceName(msg));
+
                     threadProcessingMessage(true, msgC);
 
                     processRegularMessage0(msg, nodeId);
@@ -1426,6 +1435,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             }
         };
 
+        MTC.span().addLog(() -> "Regular process queued");
+
         if (msg.topicOrdinal() == TOPIC_IO_TEST.ordinal()) {
             IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg.message();
 
@@ -1827,6 +1838,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         };
 
         try {
+            MTC.span().addLog(() -> "Ordered process queued");
+
             pools.poolForPolicy(plc).execute(c);
         }
         catch (RejectedExecutionException e) {
@@ -1882,6 +1895,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @param secSubjId Security subject that will be used to open a security session.
      */
     private void invokeListener(Byte plc, GridMessageListener lsnr, UUID nodeId, Object msg, UUID secSubjId) {
+        MTC.span().addLog(() -> "Invoke listener");
+
         Byte oldPlc = CUR_PLC.get();
 
         boolean change = !F.eq(oldPlc, plc);
@@ -2057,29 +2072,34 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         assert !async || msg instanceof GridIoUserMessage : msg; // Async execution was added only for IgniteMessaging.
         assert topicOrd >= 0 || !(topic instanceof GridTopic) : msg;
 
-        GridIoMessage ioMsg = createGridIoMessage(topic, topicOrd, msg, plc, ordered, timeout, skipOnTimeout, connIdx);
+        try (TraceSurroundings ignored = support(null)) {
+            MTC.span().addLog(() -> "Create communication msg - " + traceName(msg));
 
-        if (locNodeId.equals(node.id())) {
-            assert plc != P2P_POOL;
+            GridIoMessage ioMsg = createGridIoMessage(topic, topicOrd, msg, plc, ordered, timeout, skipOnTimeout,
+                connIdx);
 
-            CommunicationListener commLsnr = this.commLsnr;
+            if (locNodeId.equals(node.id())) {
 
-            if (commLsnr == null)
-                throw new IgniteCheckedException("Trying to send message when grid is not fully started.");
+                assert plc != P2P_POOL;
 
-            if (ordered)
-                processOrderedMessage(locNodeId, ioMsg, plc, null);
-            else if (async)
-                processRegularMessage(locNodeId, ioMsg, plc, NOOP);
-            else
-                processRegularMessage0(ioMsg, locNodeId);
+                CommunicationListener commLsnr = this.commLsnr;
 
-            if (ackC != null)
-                ackC.apply(null);
-        }
-        else {
-            if (topicOrd < 0)
-                ioMsg.topicBytes(U.marshal(marsh, topic));
+                if (commLsnr == null)
+                    throw new IgniteCheckedException("Trying to send message when grid is not fully started.");
+
+                if (ordered)
+                    processOrderedMessage(locNodeId, ioMsg, plc, null);
+                else if (async)
+                    processRegularMessage(locNodeId, ioMsg, plc, NOOP);
+                else
+                    processRegularMessage0(ioMsg, locNodeId);
+
+                if (ackC != null)
+                    ackC.apply(null);
+            }
+            else {
+                if (topicOrd < 0)
+                    ioMsg.topicBytes(U.marshal(marsh, topic));
 
                 try {
                     if ((CommunicationSpi<?>)getSpi() instanceof TcpCommunicationSpi)
@@ -2091,13 +2111,14 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                     if (e.getCause() instanceof ClusterTopologyCheckedException)
                         throw (ClusterTopologyCheckedException)e.getCause();
 
-                if (!ctx.discovery().alive(node))
-                    throw new ClusterTopologyCheckedException("Failed to send message, node left: " + node.id(), e);
+                    if (!ctx.discovery().alive(node))
+                        throw new ClusterTopologyCheckedException("Failed to send message, node left: " + node.id(), e);
 
-                throw new IgniteCheckedException("Failed to send message (node may have left the grid or " +
-                    "TCP connection cannot be established due to firewall issues) " +
-                    "[node=" + node + ", topic=" + topic +
-                    ", msg=" + msg + ", policy=" + plc + ']', e);
+                    throw new IgniteCheckedException("Failed to send message (node may have left the grid or " +
+                        "TCP connection cannot be established due to firewall issues) " +
+                        "[node=" + node + ", topic=" + topic +
+                        ", msg=" + msg + ", policy=" + plc + ']', e);
+                }
             }
         }
     }
@@ -2176,6 +2197,18 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @param plc Type of processing.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
+    public void sendToGridTopic(ClusterNode node, GridTopic topic, Message msg, byte plc)
+        throws IgniteCheckedException {
+        send(node, topic, topic.ordinal(), msg, plc, false, 0, false, null, false, UNDEFINED_CONNECTION_INDEX);
+    }
+
+    /**
+     * @param node Destination node.
+     * @param topic Topic to send the message to.
+     * @param msg Message to send.
+     * @param plc Type of processing.
+     * @throws IgniteCheckedException Thrown in case of any errors.
+     */
     public void sendToCustomTopic(ClusterNode node, Object topic, Message msg, byte plc)
         throws IgniteCheckedException {
         send(node, topic, -1, msg, plc, false, 0, false, null, false, UNDEFINED_CONNECTION_INDEX);
@@ -2186,9 +2219,10 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @param topic Topic to send the message to.
      * @param msg Message to send.
      * @param plc Type of processing.
+     * @param span Current span for tracing.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void sendToGridTopic(ClusterNode node, GridTopic topic, Message msg, byte plc)
+    public void sendToGridTopic(ClusterNode node, GridTopic topic, Message msg, byte plc, Span span)
         throws IgniteCheckedException {
         send(node, topic, topic.ordinal(), msg, plc, false, 0, false, null, false, UNDEFINED_CONNECTION_INDEX);
     }
@@ -2198,7 +2232,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @param topic Topic to send the message to.
      * @param topicOrd GridTopic enumeration ordinal.
      * @param msg Message to send.
-     * @param plc Type of processing.
+     * @param plc Type of processing.     *
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     public void sendGeneric(ClusterNode node, Object topic, int topicOrd, Message msg, byte plc)
@@ -3714,7 +3748,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
         /** */
         @GridToStringInclude
-        private final Queue<GridTuple3<GridIoMessage, Long, IgniteRunnable>> msgs = new ConcurrentLinkedDeque<>();
+        private final Queue<OrderedMessageContainer> msgs = new ConcurrentLinkedDeque<>();
 
         /** */
         private final AtomicBoolean reserved = new AtomicBoolean();
@@ -3762,7 +3796,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
             lastTs = U.currentTimeMillis();
 
-            msgs.add(F.t(msg, lastTs, msgC));
+            msgs.add(new OrderedMessageContainer(msg, lastTs, msgC, MTC.span()));
         }
 
         /** {@inheritDoc} */
@@ -3880,13 +3914,20 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         void unwind(GridMessageListener lsnr) {
             assert reserved.get();
 
-            for (GridTuple3<GridIoMessage, Long, IgniteRunnable> t = msgs.poll(); t != null; t = msgs.poll()) {
-                try {
-                    invokeListener(plc, lsnr, nodeId, t.get1().message(), secSubjId(t.get1()));
-                }
-                finally {
-                    if (t.get3() != null)
-                        t.get3().run();
+            for (OrderedMessageContainer mc = msgs.poll(); mc != null; mc = msgs.poll()) {
+                try (TraceSurroundings ignore = support(ctx.tracing().create(
+                    COMMUNICATION_ORDERED_PROCESS, mc.parentSpan))) {
+                    try {
+                        OrderedMessageContainer fmc = mc;
+
+                        MTC.span().addTag(SpanTags.MESSAGE, () -> traceName(fmc.message));
+
+                        invokeListener(plc, lsnr, nodeId, mc.message.message(), secSubjId(mc.message));
+                    }
+                    finally {
+                        if (mc.closure != null)
+                            mc.closure.run();
+                    }
                 }
             }
         }
@@ -3899,7 +3940,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             GridIoMessage msg,
             @Nullable IgniteRunnable msgC
         ) {
-            msgs.add(F.t(msg, U.currentTimeMillis(), msgC));
+            msgs.add(new OrderedMessageContainer(msg, U.currentTimeMillis(), msgC, MTC.span()));
         }
 
         /**
@@ -3932,6 +3973,37 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     }
 
     /**
+     * DTO for handling of communication message.
+     */
+    private static class OrderedMessageContainer {
+        /** */
+        GridIoMessage message;
+
+        /** */
+        long addedTime;
+
+        /** */
+        IgniteRunnable closure;
+
+        /** */
+        Span parentSpan;
+
+        /**
+         *
+         * @param msg Received message.
+         * @param addedTime Time of added to queue.
+         * @param c Message closure.
+         * @param parentSpan Span of process which added this message.
+         */
+        private OrderedMessageContainer(GridIoMessage msg, Long addedTime, IgniteRunnable c, Span parentSpan) {
+            this.message = msg;
+            this.addedTime = addedTime;
+            this.closure = c;
+            this.parentSpan = parentSpan;
+        }
+    }
+
+    /**
      *
      */
     private static class ConcurrentHashMap0<K, V> extends ConcurrentHashMap<K, V> {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index e01c0a5..3fc5e63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -19,11 +19,11 @@ package org.apache.ignite.internal.managers.communication;
 
 import java.io.Externalizable;
 import java.nio.ByteBuffer;
-
 import org.apache.ignite.internal.ExecutorAwareMessage;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest;
+import org.apache.ignite.internal.processors.tracing.messages.SpanTransport;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -35,7 +35,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Wrapper for all grid messages.
  */
-public class GridIoMessage implements TcpConnectionIndexAwareMessage {
+public class GridIoMessage implements TcpConnectionIndexAwareMessage, SpanTransport {
     /** */
     public static final Integer STRIPE_DISABLED_PART = Integer.MIN_VALUE;
 
@@ -71,6 +71,9 @@ public class GridIoMessage implements TcpConnectionIndexAwareMessage {
     /** */
     private transient int connIdx = UNDEFINED_CONNECTION_INDEX;
 
+    /** Serialized span */
+    private byte[] span;
+
     /**
      * No-op constructor to support {@link Externalizable} interface.
      * This constructor is not meant to be used for other purposes.
@@ -239,18 +242,24 @@ public class GridIoMessage implements TcpConnectionIndexAwareMessage {
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeLong("timeout", timeout))
+                if (!writer.writeByteArray("span", span))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeByteArray("topicBytes", topicBytes))
+                if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
             case 6:
+                if (!writer.writeByteArray("topicBytes", topicBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
                 if (!writer.writeInt("topicOrd", topicOrd))
                     return false;
 
@@ -302,7 +311,7 @@ public class GridIoMessage implements TcpConnectionIndexAwareMessage {
                 reader.incrementState();
 
             case 4:
-                timeout = reader.readLong("timeout");
+                span = reader.readByteArray("span");
 
                 if (!reader.isLastRead())
                     return false;
@@ -310,7 +319,7 @@ public class GridIoMessage implements TcpConnectionIndexAwareMessage {
                 reader.incrementState();
 
             case 5:
-                topicBytes = reader.readByteArray("topicBytes");
+                timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
                     return false;
@@ -318,6 +327,14 @@ public class GridIoMessage implements TcpConnectionIndexAwareMessage {
                 reader.incrementState();
 
             case 6:
+                topicBytes = reader.readByteArray("topicBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
                 topicOrd = reader.readInt("topicOrd");
 
                 if (!reader.isLastRead())
@@ -337,7 +354,17 @@ public class GridIoMessage implements TcpConnectionIndexAwareMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 7;
+        return 8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void span(byte[] span) {
+        this.span = span;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] span() {
+        return span;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoSecurityAwareMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoSecurityAwareMessage.java
index a9b62af..3c3c13e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoSecurityAwareMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoSecurityAwareMessage.java
@@ -87,7 +87,7 @@ public class GridIoSecurityAwareMessage extends GridIoMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */
@@ -105,7 +105,7 @@ public class GridIoSecurityAwareMessage extends GridIoMessage {
         }
 
         switch (writer.state()) {
-            case 7:
+            case 8:
                 if (!writer.writeUuid("secSubjId", secSubjId))
                     return false;
 
@@ -127,7 +127,7 @@ public class GridIoSecurityAwareMessage extends GridIoMessage {
             return false;
 
         switch (reader.state()) {
-            case 7:
+            case 8:
                 secSubjId = reader.readUuid("secSubjId");
 
                 if (!reader.isLastRead())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TraceRunnable.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TraceRunnable.java
new file mode 100644
index 0000000..6ce4f9c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TraceRunnable.java
@@ -0,0 +1,63 @@
+/*
+ * 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.managers.communication;
+
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanType;
+import org.apache.ignite.internal.processors.tracing.Tracing;
+
+/**
+ * Wrapper of {@link Runnable} which incject tracing to execution.
+ */
+public abstract class TraceRunnable implements Runnable {
+    /** */
+    private final Tracing tracing;
+
+    /** SpanType of the new span. */
+    private final SpanType spanType;
+
+    /** Parent span from which new span should be created. */
+    private final Span parent;
+
+    /**
+     * @param tracing Tracing processor.
+     * @param spanType Span type to create.
+     */
+    protected TraceRunnable(Tracing tracing, SpanType spanType) {
+        this.tracing = tracing;
+        this.spanType = spanType;
+        parent = MTC.span();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run() {
+        Span span = tracing.create(spanType, parent);
+
+        try (TraceSurroundings ignore = MTC.support(span.equals(NoopSpan.INSTANCE) ? parent : span)) {
+            execute();
+        }
+    }
+
+    /**
+     * Main code to execution.
+     */
+    public abstract void execute();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 866b8fb..40e1284 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -88,13 +88,13 @@ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.processors.cluster.IGridClusterStateProcessor;
 import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.processors.tracing.messages.SpanContainer;
 import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
-import org.apache.ignite.internal.util.lang.GridTuple6;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
@@ -119,8 +119,8 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.apache.ignite.spi.discovery.DiscoveryNotification;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
 import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
 import org.apache.ignite.spi.discovery.DiscoverySpiListener;
@@ -519,26 +519,20 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 }
             }
 
-            @Override public IgniteFuture<?> onDiscovery(
-                final int type,
-                final long topVer,
-                final ClusterNode node,
-                final Collection<ClusterNode> topSnapshot,
-                final Map<Long, Collection<ClusterNode>> snapshots,
-                @Nullable DiscoverySpiCustomMessage spiCustomMsg
-            ) {
+            /** {@inheritDoc} */
+            @Override public IgniteFuture<?> onDiscovery(DiscoveryNotification notification) {
                 GridFutureAdapter<?> notificationFut = new GridFutureAdapter<>();
 
                 discoNtfWrk.submit(notificationFut, () -> {
                     synchronized (discoEvtMux) {
-                        onDiscovery0(type, topVer, node, topSnapshot, snapshots, spiCustomMsg);
+                        onDiscovery0(notification);
                     }
                 });
 
                 IgniteFuture<?> fut = new IgniteFutureImpl<>(notificationFut);
 
                 //TODO could be optimized with more specific conditions.
-                switch (type) {
+                switch (notification.type()) {
                     case EVT_NODE_JOINED:
                     case EVT_NODE_LEFT:
                     case EVT_NODE_FAILED:
@@ -555,31 +549,23 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             }
 
             /**
-             * @param type Event type.
-             * @param topVer Event topology version.
-             * @param node Event node.
-             * @param topSnapshot Topology snapsjot.
-             * @param snapshots Topology snapshots history.
-             * @param spiCustomMsg Custom event.
+             * @param notification Notification.
              */
-            private void onDiscovery0(
-                final int type,
-                final long topVer,
-                final ClusterNode node,
-                final Collection<ClusterNode> topSnapshot,
-                final Map<Long, Collection<ClusterNode>> snapshots,
-                @Nullable DiscoverySpiCustomMessage spiCustomMsg
-            ) {
-                DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
-                    : ((CustomMessageWrapper)spiCustomMsg).delegate();
-
-                if (skipMessage(type, customMsg))
+            private void onDiscovery0(DiscoveryNotification notification) {
+                int type = notification.type();
+                ClusterNode node = notification.getNode();
+                long topVer = notification.getTopVer();
+
+                DiscoveryCustomMessage customMsg = notification.getCustomMsgData() == null ? null
+                    : ((CustomMessageWrapper)notification.getCustomMsgData()).delegate();
+
+                if (skipMessage(notification.type(), customMsg))
                     return;
 
                 final ClusterNode locNode = localNode();
 
-                if (snapshots != null)
-                    topHist = snapshots;
+                if (notification.getTopHist() != null)
+                    topHist = notification.getTopHist();
 
                 boolean verChanged;
 
@@ -674,14 +660,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                             nextTopVer,
                             ctx.state().clusterState(),
                             locNode,
-                            topSnapshot);
+                            notification.getTopSnapshot());
                     }
                     else if (customMsg instanceof ChangeGlobalStateMessage) {
                         discoCache = createDiscoCache(
                             nextTopVer,
                             ctx.state().pendingState((ChangeGlobalStateMessage)customMsg),
                             locNode,
-                            topSnapshot);
+                            notification.getTopSnapshot());
                     }
                     else
                         discoCache = customMsg.createDiscoCache(GridDiscoveryManager.this, nextTopVer, snapshot.discoCache);
@@ -702,7 +688,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     if (type == EVT_NODE_LEFT || type == EVT_NODE_FAILED || type == EVT_NODE_JOINED) {
                         boolean discoCacheRecalculationRequired = ctx.state().autoAdjustInMemoryClusterState(
                             node.id(),
-                            topSnapshot,
+                            notification.getTopSnapshot(),
                             discoCache,
                             topVer,
                             minorTopVer
@@ -713,7 +699,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                                 nextTopVer,
                                 ctx.state().clusterState(),
                                 locNode,
-                                topSnapshot
+                                notification.getTopSnapshot()
                             );
 
                             discoCacheHist.put(nextTopVer, discoCache);
@@ -755,7 +741,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     discoEvt.eventNode(node);
                     discoEvt.type(EVT_NODE_JOINED);
 
-                    discoEvt.topologySnapshot(topVer, new ArrayList<>(F.view(topSnapshot, FILTER_NOT_DAEMON)));
+                    discoEvt.topologySnapshot(topVer, new ArrayList<>(F.view(notification.getTopSnapshot(), FILTER_NOT_DAEMON)));
+
+                    if (notification.getSpanContainer() != null)
+                        discoEvt.span(notification.getSpanContainer().span());
 
                     discoWrk.discoCache = discoCache;
 
@@ -837,7 +826,17 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                             try {
                                 fut.get();
 
-                                discoWrk.addEvent(EVT_CLIENT_NODE_RECONNECTED, nextTopVer, node, discoCache0, topSnapshot, null);
+                                discoWrk.addEvent(
+                                    new NotificationEvent(
+                                        EVT_CLIENT_NODE_RECONNECTED,
+                                        nextTopVer,
+                                        node,
+                                        discoCache0,
+                                        notification.getTopSnapshot(),
+                                        null,
+                                        notification.getSpanContainer()
+                                    )
+                                );
                             }
                             catch (IgniteException ignore) {
                                 // No-op.
@@ -849,10 +848,29 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 }
 
                 if (type == EVT_CLIENT_NODE_DISCONNECTED || type == EVT_NODE_SEGMENTED || !ctx.clientDisconnected())
-                    discoWrk.addEvent(type, nextTopVer, node, discoCache, topSnapshot, customMsg);
+                    discoWrk.addEvent(
+                        new NotificationEvent(
+                            type,
+                            nextTopVer,
+                            node, discoCache,
+                            notification.getTopSnapshot(),
+                            customMsg,
+                            notification.getSpanContainer()
+                        )
+                    );
 
                 if (stateFinishMsg != null)
-                    discoWrk.addEvent(EVT_DISCOVERY_CUSTOM_EVT, nextTopVer, node, discoCache, topSnapshot, stateFinishMsg);
+                    discoWrk.addEvent(
+                        new NotificationEvent(
+                            EVT_DISCOVERY_CUSTOM_EVT,
+                            nextTopVer,
+                            node,
+                            discoCache,
+                            notification.getTopSnapshot(),
+                            stateFinishMsg,
+                            notification.getSpanContainer()
+                        )
+                    );
 
                 if (type == EVT_CLIENT_NODE_DISCONNECTED)
                     discoWrk.awaitDisconnectEvent();
@@ -2184,26 +2202,38 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         @Nullable Set<String> cachesToClose) {
         // Prevent race when discovery message was processed, but was passed to discoWrk.
         synchronized (discoEvtMux) {
-            discoWrk.addEvent(EVT_DISCOVERY_CUSTOM_EVT,
-                AffinityTopologyVersion.NONE,
-                localNode(),
-                null,
-                Collections.<ClusterNode>emptyList(),
-                new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose));
+            discoWrk.addEvent(
+                new NotificationEvent(
+                    EVT_DISCOVERY_CUSTOM_EVT,
+                    AffinityTopologyVersion.NONE,
+                    localNode(),
+                    null,
+                    Collections.<ClusterNode>emptyList(),
+                    new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose),
+                    null
+                )
+            );
         }
     }
 
     /**
-     * @param discoCache
-     * @param node
+     * Adds metrics update event to discovery worker queue.
+     *
+     * @param discoCache Discovery cache.
+     * @param node Event node.
      */
     public void metricsUpdateEvent(DiscoCache discoCache, ClusterNode node) {
-        discoWrk.addEvent(EVT_NODE_METRICS_UPDATED,
-            discoCache.version(),
-            node,
-            discoCache,
-            discoCache.nodeMap.values(),
-            null);
+        discoWrk.addEvent(
+            new NotificationEvent(
+                EVT_NODE_METRICS_UPDATED,
+                discoCache.version(),
+                node,
+                discoCache,
+                discoCache.nodeMap.values(),
+                null,
+                null
+            )
+        );
     }
 
     /**
@@ -2579,16 +2609,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                         Collection<ClusterNode> locNodeOnlyTop = Collections.singleton(node);
 
-                        discoWrk.addEvent(EVT_NODE_SEGMENTED,
-                            AffinityTopologyVersion.NONE,
-                            node,
-                            createDiscoCache(
+                        discoWrk.addEvent(
+                            new NotificationEvent(
+                                EVT_NODE_SEGMENTED,
                                 AffinityTopologyVersion.NONE,
-                                ctx.state().clusterState(),
                                 node,
-                                locNodeOnlyTop),
-                            locNodeOnlyTop,
-                            null);
+                                createDiscoCache(
+                                    AffinityTopologyVersion.NONE,
+                                    ctx.state().clusterState(),
+                                    node,
+                                    locNodeOnlyTop),
+                                locNodeOnlyTop,
+                                null,
+                                null
+                            )
+                        );
 
                         lastSegChkRes.set(false);
                     }
@@ -2712,14 +2747,66 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
     }
 
+    /**
+     * Internal notification event.
+     */
+    private static class NotificationEvent {
+        /** Type. */
+        int type;
+
+        /** Topology version. */
+        AffinityTopologyVersion topVer;
+
+        /** Node. */
+        ClusterNode node;
+
+        /** Disco cache. */
+        DiscoCache discoCache;
+
+        /** Topology snapshot. */
+        Collection<ClusterNode> topSnapshot;
+
+        /** Data. */
+        @Nullable DiscoveryCustomMessage data;
+
+        /** Span container. */
+        SpanContainer spanContainer;
+
+        /**
+         * @param type Type.
+         * @param topVer Topology version.
+         * @param node Node.
+         * @param discoCache Disco cache.
+         * @param topSnapshot Topology snapshot.
+         * @param data Data.
+         * @param spanContainer Span container.
+         */
+        public NotificationEvent(
+            int type,
+            AffinityTopologyVersion topVer,
+            ClusterNode node,
+            DiscoCache discoCache,
+            Collection<ClusterNode> topSnapshot,
+            @Nullable DiscoveryCustomMessage data,
+            SpanContainer spanContainer
+        ) {
+            this.type = type;
+            this.topVer = topVer;
+            this.node = node;
+            this.discoCache = discoCache;
+            this.topSnapshot = topSnapshot;
+            this.data = data;
+            this.spanContainer = spanContainer;
+        }
+    }
+
     /** Worker for discovery events. */
     private class DiscoveryWorker extends GridWorker {
         /** */
         private DiscoCache discoCache;
 
         /** Event queue. */
-        private final BlockingQueue<GridTuple6<Integer, AffinityTopologyVersion, ClusterNode,
-            DiscoCache, Collection<ClusterNode>, DiscoveryCustomMessage>> evts = new LinkedBlockingQueue<>();
+        private final BlockingQueue<NotificationEvent> evts = new LinkedBlockingQueue<>();
 
         /** Restart process handler. */
         private final RestartProcessFailureHandler restartProcHnd = new RestartProcessFailureHandler();
@@ -2755,7 +2842,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @param topSnapshot Topology snapshot.
          */
         @SuppressWarnings("RedundantTypeArguments")
-        private void recordEvent(int type, long topVer, ClusterNode node, DiscoCache discoCache, Collection<ClusterNode> topSnapshot) {
+        private void recordEvent(int type, long topVer, ClusterNode node, DiscoCache discoCache, Collection<ClusterNode> topSnapshot, @Nullable SpanContainer spanContainer) {
             assert node != null;
 
             if (ctx.event().isRecordable(type)) {
@@ -2765,6 +2852,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 evt.eventNode(node);
                 evt.type(type);
                 evt.topologySnapshot(topVer, U.<ClusterNode, ClusterNode>arrayList(topSnapshot, FILTER_NOT_DAEMON));
+                evt.span(spanContainer != null ? spanContainer.span() : null);
 
                 if (type == EVT_NODE_METRICS_UPDATED)
                     evt.message("Metrics were updated");
@@ -2795,27 +2883,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
-         * @param type Event type.
-         * @param topVer Topology version.
-         * @param node Node.
-         * @param discoCache Discovery cache.
-         * @param topSnapshot Topology snapshot.
-         * @param data Custom message.
+         * @param notificationEvt Notification event.
          */
-        void addEvent(
-            int type,
-            AffinityTopologyVersion topVer,
-            ClusterNode node,
-            DiscoCache discoCache,
-            Collection<ClusterNode> topSnapshot,
-            @Nullable DiscoveryCustomMessage data
-        ) {
-            assert node != null : data;
+        void addEvent(NotificationEvent notificationEvt) {
+            assert notificationEvt.node != null : notificationEvt.data;
 
-            if (type == EVT_CLIENT_NODE_DISCONNECTED)
+            if (notificationEvt.type == EVT_CLIENT_NODE_DISCONNECTED)
                 discoWrk.disconnectEvtFut = new GridFutureAdapter();
 
-            evts.add(new GridTuple6<>(type, topVer, node, discoCache, topSnapshot, data));
+            evts.add(notificationEvt);
         }
 
         /** {@inheritDoc} */
@@ -2846,8 +2922,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         /** @throws InterruptedException If interrupted. */
         private void body0() throws InterruptedException {
-            GridTuple6<Integer, AffinityTopologyVersion, ClusterNode, DiscoCache, Collection<ClusterNode>,
-                DiscoveryCustomMessage> evt;
+            NotificationEvent evt;
 
             blockingSectionBegin();
 
@@ -2858,21 +2933,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 blockingSectionEnd();
             }
 
-            int type = evt.get1();
+            int type = evt.type;
 
-            AffinityTopologyVersion topVer = evt.get2();
+            AffinityTopologyVersion topVer = evt.topVer;
 
             if (type == EVT_NODE_METRICS_UPDATED && (discoCache == null || topVer.compareTo(discoCache.version()) < 0))
                 return;
 
-            ClusterNode node = evt.get3();
+            ClusterNode node = evt.node;
 
             boolean isDaemon = node.isDaemon();
 
             boolean segmented = false;
 
-            if (evt.get4() != null)
-                discoCache = evt.get4();
+            if (evt.discoCache != null)
+                discoCache = evt.discoCache;
 
             switch (type) {
                 case EVT_NODE_JOINED: {
@@ -2994,17 +3069,18 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         customEvt.node(ctx.discovery().localNode());
                         customEvt.eventNode(node);
                         customEvt.type(type);
-                        customEvt.topologySnapshot(topVer.topologyVersion(), evt.get5());
+                        customEvt.topologySnapshot(topVer.topologyVersion(), evt.topSnapshot);
                         customEvt.affinityTopologyVersion(topVer);
-                        customEvt.customMessage(evt.get6());
+                        customEvt.customMessage(evt.data);
+                        customEvt.span(evt.spanContainer != null ? evt.spanContainer.span() : null);
 
-                        if (evt.get4() == null) {
-                            assert discoCache != null : evt.get6();
+                        if (evt.discoCache == null) {
+                            assert discoCache != null : evt.data;
 
-                            evt.set4(discoCache);
+                            evt.discoCache = discoCache;
                         }
 
-                        ctx.event().record(customEvt, evt.get4());
+                        ctx.event().record(customEvt, evt.discoCache);
                     }
 
                     return;
@@ -3018,7 +3094,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     assert false : "Invalid discovery event: " + type;
             }
 
-            recordEvent(type, topVer.topologyVersion(), node, evt.get4(), evt.get5());
+            recordEvent(type, topVer.topologyVersion(), node, evt.discoCache, evt.topSnapshot, evt.spanContainer);
 
             if (segmented)
                 onSegmentation();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/tracing/GridTracingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/tracing/GridTracingManager.java
new file mode 100644
index 0000000..94bffa6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/tracing/GridTracingManager.java
@@ -0,0 +1,534 @@
+/*
+ * 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.managers.tracing;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.GridManagerAdapter;
+import org.apache.ignite.internal.processors.tracing.DeferredSpan;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.NoopTracing;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanImpl;
+import org.apache.ignite.internal.processors.tracing.SpanTags;
+import org.apache.ignite.internal.processors.tracing.SpanType;
+import org.apache.ignite.internal.processors.tracing.Tracing;
+import org.apache.ignite.internal.processors.tracing.configuration.GridTracingConfigurationManager;
+import org.apache.ignite.internal.processors.tracing.messages.TraceableMessagesHandler;
+import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.logger.NullLogger;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.tracing.NoopTracingSpi;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.SpiSpecificSpan;
+import org.apache.ignite.spi.tracing.TracingConfigurationCoordinates;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.apache.ignite.spi.tracing.TracingConfigurationParameters;
+import org.apache.ignite.spi.tracing.TracingSpi;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.tracing.SpanTags.NODE;
+import static org.apache.ignite.internal.util.GridClientByteUtils.bytesToInt;
+import static org.apache.ignite.internal.util.GridClientByteUtils.bytesToShort;
+import static org.apache.ignite.internal.util.GridClientByteUtils.intToBytes;
+import static org.apache.ignite.internal.util.GridClientByteUtils.shortToBytes;
+import static org.apache.ignite.spi.tracing.TracingConfigurationParameters.SAMPLING_RATE_NEVER;
+
+/**
+ * Tracing Manager.
+ */
+public class GridTracingManager extends GridManagerAdapter<TracingSpi> implements Tracing {
+    /** */
+    private static final int SPECIAL_FLAGS_OFF = 0;
+
+    /** */
+    private static final int SPI_TYPE_OFF = SPECIAL_FLAGS_OFF + 1;
+
+    /** */
+    private static final int MAJOR_PROTOCOL_VERSION_OFF = SPI_TYPE_OFF + 1;
+
+    /** */
+    private static final int MINOR_PROTOCOL_VERSION_OFF = MAJOR_PROTOCOL_VERSION_OFF + 1;
+
+    /** */
+    private static final int SPI_SPECIFIC_SERIALIZED_SPAN_BYTES_LENGTH_OFF = MINOR_PROTOCOL_VERSION_OFF + 1;
+
+    /** */
+    private static final int SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF = SPI_SPECIFIC_SERIALIZED_SPAN_BYTES_LENGTH_OFF + 4;
+
+    /** */
+    private static final int PARENT_SPAN_TYPE_BYTES_LENGTH = 4;
+
+    /** */
+    private static final int INCLUDED_SCOPES_SIZE_BYTE_LENGTH = 4;
+
+    /** */
+    private static final int SCOPE_INDEX_BYTE_LENGTH = 2;
+
+    /** */
+    private static final int SPI_SPECIFIC_SERIALIZED_SPAN_BYTES_LENGTH = 4;
+
+    /** Traceable messages handler. */
+    private final TraceableMessagesHandler msgHnd;
+
+    /** Tracing configuration */
+    private final TracingConfigurationManager tracingConfiguration;
+
+    /**
+     * Major span serialization protocol version.
+     * Within same major protocol version span serialization should be backward compatible.
+     */
+    private static final byte MAJOR_PROTOCOL_VERSION = 0;
+
+    /** Minor span serialization protocol version. */
+    private static final byte MINOR_PROTOCOL_VERSION = 0;
+
+    /** Noop traceable message handler. */
+    private static final TraceableMessagesHandler NOOP_TRACEABLE_MSG_HANDLER =
+        new TraceableMessagesHandler(new NoopTracing(), new NullLogger());
+
+    /** Flag that indicates that noop tracing spi is used. */
+    private boolean noop = true;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     * @param useNoopTracingSpi Flag that signals that NoOp tracing spi should be used instead of the one,
+     * specified in the context. It's a part of the failover logic that is suitable if an exception is thrown
+     * when the manager starts.
+     */
+    public GridTracingManager(GridKernalContext ctx, boolean useNoopTracingSpi) {
+        super(ctx, useNoopTracingSpi ? new NoopTracingSpi() : ctx.config().getTracingSpi());
+
+        msgHnd = new TraceableMessagesHandler(this, ctx.log(GridTracingManager.class));
+
+        tracingConfiguration = new GridTracingConfigurationManager(ctx);
+    }
+
+    /**
+     * @throws IgniteCheckedException Thrown in case of any errors.
+     */
+    @Override public void start() throws IgniteCheckedException {
+        try {
+            startSpi();
+
+            noop = getSpi() instanceof NoopTracingSpi;
+        }
+        catch (IgniteSpiException e) {
+            log.warning("Failed to start tracing processor with spi: " + getSpi().getName()
+                + ". Noop implementation will be used instead.", e);
+
+            throw e;
+        }
+
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /**
+     * @throws IgniteCheckedException Thrown in case of any errors.
+     */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        stopSpi();
+
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /**
+     * Adds tags with information about local node to given {@code span}.
+     *
+     * @param span Span.
+     * @return Span enriched by local node information.
+     */
+    private Span enrichWithLocalNodeParameters(@Nullable Span span) {
+        if (span == null)
+            return null;
+
+        span.addTag(SpanTags.NODE_ID, () -> ctx.localNodeId().toString());
+        span.addTag(SpanTags.tag(NODE, SpanTags.NAME), ctx::igniteInstanceName);
+
+        ClusterNode locNode = ctx.discovery().localNode();
+        if (locNode != null && locNode.consistentId() != null)
+            span.addTag(SpanTags.tag(NODE, SpanTags.CONSISTENT_ID), () -> locNode.consistentId().toString());
+
+        return span;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span create(@NotNull SpanType spanType, @Nullable Span parentSpan) {
+        // Optimization for noop spi.
+        if (noop)
+            return NoopSpan.INSTANCE;
+
+        // Optimization for zero sampling rate == 0.
+        if ((parentSpan == NoopSpan.INSTANCE || parentSpan == null) &&
+            tracingConfiguration.get(new TracingConfigurationCoordinates.Builder(spanType.scope()).build()).
+                samplingRate() == SAMPLING_RATE_NEVER)
+            return NoopSpan.INSTANCE;
+
+        return enrichWithLocalNodeParameters(
+            generateSpan(
+                parentSpan,
+                spanType,
+                null));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span create(@NotNull SpanType spanType, @Nullable byte[] serializedParentSpan) {
+        // Optimization for noop spi.
+        if (noop)
+            return NoopSpan.INSTANCE;
+
+        // Optimization for zero sampling rate == 0.
+        if ((serializedParentSpan.length == 0 || serializedParentSpan == null) &&
+            tracingConfiguration.get(new TracingConfigurationCoordinates.Builder(spanType.scope()).build()).
+                samplingRate() == SAMPLING_RATE_NEVER)
+            return NoopSpan.INSTANCE;
+
+        // 1 byte: special flags;
+        // 1 bytes: spi type;
+        // 2 bytes: major protocol version;
+        // 2 bytes: minor protocol version;
+        // 4 bytes: spi specific serialized span length;
+        // n bytes: spi specific serialized span body;
+        // 4 bytes: span type
+        // 4 bytes included scopes size;
+        // 2 * included scopes size: included scopes items one by one;
+
+        Span span;
+
+        try {
+            if (serializedParentSpan == null || serializedParentSpan.length == 0)
+                return create(spanType, NoopSpan.INSTANCE);
+
+            // First byte of the serializedSpan is reserved for special flags - it's not used right now.
+
+            // Deserialize and compare spi types. If they don't match (span was serialized with another spi) then
+            // propagate serializedSpan as DeferredSpan.
+            if (serializedParentSpan[SPI_TYPE_OFF] != getSpi().type())
+                return new DeferredSpan(serializedParentSpan);
+
+            // Deserialize and check major protocol version,
+            // cause protocol might be incompatible in case of different protocol versions -
+            // propagate serializedSpan as DeferredSpan.
+            if (serializedParentSpan[MAJOR_PROTOCOL_VERSION_OFF] != MAJOR_PROTOCOL_VERSION)
+                return new DeferredSpan(serializedParentSpan);
+
+            // Deserialize and check minor protocol version.
+            // within the scope of the same major protocol version, protocol should be backwards compatible
+            byte minProtoVer = serializedParentSpan[MINOR_PROTOCOL_VERSION_OFF];
+
+            // Deserialize spi specific span size.
+            int spiSpecificSpanSize = bytesToInt(
+                Arrays.copyOfRange(
+                    serializedParentSpan,
+                    SPI_SPECIFIC_SERIALIZED_SPAN_BYTES_LENGTH_OFF,
+                    SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF),
+                0);
+
+            SpanType parentSpanType = null;
+
+            Set<Scope> includedScopes = new HashSet<>();
+
+            // Fall through.
+            switch (minProtoVer) {
+                case 0 : {
+                    // Deserialize parent span type.
+                    parentSpanType = SpanType.fromIndex(
+                        bytesToInt(
+                            Arrays.copyOfRange(
+                                serializedParentSpan,
+                                SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + spiSpecificSpanSize,
+                                SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                                    spiSpecificSpanSize),
+                            0));
+
+                    // Deserialize included scopes size.
+                    int includedScopesSize = bytesToInt(
+                        Arrays.copyOfRange(
+                            serializedParentSpan,
+                            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                                spiSpecificSpanSize,
+                            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                                INCLUDED_SCOPES_SIZE_BYTE_LENGTH + spiSpecificSpanSize),
+                        0);
+
+                    // Deserialize included scopes one by one.
+                    for (int i = 0; i < includedScopesSize; i++) {
+                        includedScopes.add(Scope.fromIndex(
+                            bytesToShort(
+                                Arrays.copyOfRange(
+                                    serializedParentSpan,
+                                    SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                                        INCLUDED_SCOPES_SIZE_BYTE_LENGTH + spiSpecificSpanSize +
+                                        i * SCOPE_INDEX_BYTE_LENGTH,
+                                    SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                                        INCLUDED_SCOPES_SIZE_BYTE_LENGTH + SCOPE_INDEX_BYTE_LENGTH +
+                                        spiSpecificSpanSize + i * SCOPE_INDEX_BYTE_LENGTH),
+                                0)));
+                    }
+                }
+            }
+
+            assert parentSpanType != null;
+
+            // If there's is parent span and parent span supports given scope then...
+            if (parentSpanType.scope() == spanType.scope() || includedScopes.contains(spanType.scope())) {
+                // create new span as child span for parent span, using parents span included scopes.
+
+                Set<Scope> mergedIncludedScopes = new HashSet<>(includedScopes);
+                mergedIncludedScopes.add(parentSpanType.scope());
+                mergedIncludedScopes.remove(spanType.scope());
+
+                span = new SpanImpl(
+                    getSpi().create(
+                        spanType.spanName(),
+                        Arrays.copyOfRange(
+                            serializedParentSpan,
+                            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF,
+                            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + spiSpecificSpanSize)),
+                    spanType,
+                    mergedIncludedScopes);
+            }
+            else {
+                // do nothing;
+                return new DeferredSpan(serializedParentSpan);
+                // "suppress" parent span for a while, create new span as separate one.
+                // return spi.create(trace, null, includedScopes);
+            }
+        }
+        catch (Exception e) {
+            LT.warn(log, "Failed to create span from serialized value " +
+                "[serializedValue=" + Arrays.toString(serializedParentSpan) + "]");
+
+            span = NoopSpan.INSTANCE;
+        }
+
+        return enrichWithLocalNodeParameters(span);
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull Span create(
+        @NotNull SpanType spanType,
+        @Nullable Span parentSpan,
+        @Nullable String lb
+    ) {
+        // Optimization for noop spi.
+        if (noop)
+            return NoopSpan.INSTANCE;
+
+        // Optimization for zero sampling rate == 0.
+        if ((parentSpan == NoopSpan.INSTANCE || parentSpan == null) &&
+            tracingConfiguration.get(
+                new TracingConfigurationCoordinates.Builder(spanType.scope()).withLabel(lb).build()).
+                samplingRate() == SAMPLING_RATE_NEVER)
+            return NoopSpan.INSTANCE;
+
+        return enrichWithLocalNodeParameters(
+            generateSpan(
+                parentSpan,
+                spanType,
+                lb));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public byte[] serialize(@NotNull Span span) {
+        // Optimization for noop spi.
+        if (noop)
+            return NoopTracing.NOOP_SERIALIZED_SPAN;
+
+        // Optimization for NoopSpan.
+        if (span == NoopSpan.INSTANCE)
+            return NoopTracing.NOOP_SERIALIZED_SPAN;
+
+        // 1 byte: special flags;
+        // 1 bytes: spi type;
+        // 2 bytes: major protocol version;
+        // 2 bytes: minor protocol version;
+        // 4 bytes: spi specific serialized span length;
+        // n bytes: spi specific serialized span body;
+        // 4 bytes: span type
+        // 4 bytes included scopes size;
+        // 2 * included scopes size: included scopes items one by one;
+
+        if (span instanceof DeferredSpan)
+            return ((DeferredSpan)span).serializedSpan();
+
+        // Spi specific serialized span.
+        byte[] spiSpecificSerializedSpan = getSpi().serialize(((SpanImpl)span).spiSpecificSpan());
+
+        int serializedSpanLen = SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+            INCLUDED_SCOPES_SIZE_BYTE_LENGTH + spiSpecificSerializedSpan.length + SCOPE_INDEX_BYTE_LENGTH *
+            span.includedScopes().size();
+
+        byte[] serializedSpanBytes = new byte[serializedSpanLen];
+
+        // Skip special flags bytes.
+
+        // Spi type idx.
+        serializedSpanBytes[SPI_TYPE_OFF] = getSpi().type();
+
+        // Major protocol version;
+        serializedSpanBytes[MAJOR_PROTOCOL_VERSION_OFF] = MAJOR_PROTOCOL_VERSION;
+
+        // Minor protocol version;
+        serializedSpanBytes[MINOR_PROTOCOL_VERSION_OFF] = MINOR_PROTOCOL_VERSION;
+
+        // Spi specific serialized span length.
+        System.arraycopy(
+            intToBytes(spiSpecificSerializedSpan.length),
+            0,
+            serializedSpanBytes,
+            SPI_SPECIFIC_SERIALIZED_SPAN_BYTES_LENGTH_OFF,
+            SPI_SPECIFIC_SERIALIZED_SPAN_BYTES_LENGTH);
+
+        // Spi specific span.
+        System.arraycopy(
+            spiSpecificSerializedSpan,
+            0,
+            serializedSpanBytes,
+            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF,
+            spiSpecificSerializedSpan.length);
+
+        // Span type.
+        System.arraycopy(
+            intToBytes(span.type().index()),
+            0,
+            serializedSpanBytes,
+            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + spiSpecificSerializedSpan.length,
+            PARENT_SPAN_TYPE_BYTES_LENGTH );
+
+        assert span.includedScopes() != null;
+
+        // Included scope size
+        System.arraycopy(
+            intToBytes(span.includedScopes().size()),
+            0,
+            serializedSpanBytes,
+            SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                spiSpecificSerializedSpan.length,
+            INCLUDED_SCOPES_SIZE_BYTE_LENGTH);
+
+        int includedScopesCnt = 0;
+
+        if (!span.includedScopes().isEmpty()) {
+            for (Scope includedScope : span.includedScopes()) {
+                System.arraycopy(
+                    shortToBytes(includedScope.idx()),
+                    0,
+                    serializedSpanBytes,
+                    SPI_SPECIFIC_SERIALIZED_SPAN_BODY_OFF + PARENT_SPAN_TYPE_BYTES_LENGTH +
+                        INCLUDED_SCOPES_SIZE_BYTE_LENGTH + spiSpecificSerializedSpan.length +
+                        SCOPE_INDEX_BYTE_LENGTH * includedScopesCnt++,
+                    SCOPE_INDEX_BYTE_LENGTH);
+            }
+        }
+
+        return serializedSpanBytes;
+    }
+
+    /**
+     * Generates child span if it's possible due to parent/child included scopes, otherwise returns patent span as is.
+     * @param parentSpan Parent span.
+     * @param spanTypeToCreate Span type to create.
+     * @param lb Label.
+     */
+    @SuppressWarnings("unchecked")
+    private @NotNull Span generateSpan(
+        @Nullable Span parentSpan,
+        @NotNull SpanType spanTypeToCreate,
+        @Nullable String lb
+    ) {
+        if (parentSpan instanceof DeferredSpan)
+            return create(spanTypeToCreate, ((DeferredSpan)parentSpan).serializedSpan());
+
+        if (parentSpan == NoopSpan.INSTANCE || parentSpan == null) {
+            if (spanTypeToCreate.rootSpan()) {
+                // Get tracing configuration.
+                TracingConfigurationParameters tracingConfigurationParameters = tracingConfiguration.get(
+                    new TracingConfigurationCoordinates.Builder(spanTypeToCreate.scope()).withLabel(lb).build());
+
+                return shouldSample(tracingConfigurationParameters.samplingRate()) ?
+                    new SpanImpl(
+                        getSpi().create(
+                            spanTypeToCreate.spanName(),
+                            (SpiSpecificSpan)null),
+                        spanTypeToCreate,
+                        tracingConfigurationParameters.includedScopes()) :
+                    NoopSpan.INSTANCE;
+            }
+            else
+                return NoopSpan.INSTANCE;
+        }
+        else {
+            // If there's is parent span and parent span supports given scope then...
+            if (parentSpan.isChainable(spanTypeToCreate.scope())) {
+                // create new span as child span for parent span, using parents span included scopes.
+
+                Set<Scope> mergedIncludedScopes = new HashSet<>(parentSpan.includedScopes());
+
+                mergedIncludedScopes.add(parentSpan.type().scope());
+                mergedIncludedScopes.remove(spanTypeToCreate.scope());
+
+                return new SpanImpl(
+                    getSpi().create(
+                        spanTypeToCreate.spanName(),
+                        ((SpanImpl)parentSpan).spiSpecificSpan()),
+                    spanTypeToCreate,
+                    mergedIncludedScopes);
+            }
+            else {
+                // do nothing;
+                return NoopSpan.INSTANCE;
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public TraceableMessagesHandler messages() {
+        // Optimization for noop spi.
+        if (noop)
+            return NOOP_TRACEABLE_MSG_HANDLER;
+
+        return msgHnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull TracingConfigurationManager configuration() {
+        return tracingConfiguration;
+    }
+
+    /**
+     * @param samlingRate Sampling rate.
+     * @return {@code true} if according to given sampling-rate span should be sampled.
+     */
+    private boolean shouldSample(double samlingRate) {
+        if (samlingRate == SAMPLING_RATE_NEVER)
+            return false;
+
+        return Math.random() <= samlingRate;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 7c4371e..61a57c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.GridPartitionStateMap;
@@ -89,6 +90,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
+import static org.apache.ignite.internal.processors.tracing.SpanType.AFFINITY_CALCULATION;
 
 /**
  *
@@ -2144,6 +2146,9 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 if (cache.affinity().lastVersion().equals(evts.topologyVersion()))
                     return;
 
+                Span affCalcSpan = cctx.kernalContext().tracing().create(AFFINITY_CALCULATION, fut.span())
+                    .addTag("cache.group", desc::cacheOrGroupName);
+
                 boolean latePrimary = cache.rebalanceEnabled;
 
                 boolean grpAdded = evts.nodeJoined(desc.receivedFrom());
@@ -2170,6 +2175,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
                 cctx.exchange().exchangerUpdateHeartbeat();
 
+                affCalcSpan.end();
+
                 fut.timeBag().finishLocalStage("Affinity initialization (node join) " +
                     "[grp=" + desc.cacheOrGroupName() + ", crd=" + crd + "]");
             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 070ffdd..8b068da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -4363,7 +4363,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     !ctx.skipStore(),
                     ctx.mvccEnabled(),
                     0,
-                    null
+                    null,
+                    false
                 );
 
                 assert tx != null;
@@ -4481,7 +4482,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     !skipStore,
                     ctx.mvccEnabled(),
                     0,
-                    null);
+                    null,
+                    false);
 
                 return asyncOp(tx, op, opCtx, /*retry*/false);
             }
@@ -5493,7 +5495,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 opCtx == null || !opCtx.skipStore(),
                 ctx.mvccEnabled(),
                 0,
-                null);
+                null,
+                false);
 
             IgniteInternalFuture<T> fut = asyncOp(tx, op, opCtx, retry);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index d569151..26be98f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -122,6 +122,8 @@ import org.apache.ignite.internal.processors.metric.impl.BooleanMetricImpl;
 import org.apache.ignite.internal.processors.metric.impl.HistogramMetricImpl;
 import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanTags;
 import org.apache.ignite.internal.util.GridListSet;
 import org.apache.ignite.internal.util.GridPartitionStateMap;
 import org.apache.ignite.internal.util.GridStringBuilder;
@@ -181,6 +183,7 @@ import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME
 import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION;
 import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION_HISTOGRAM;
 import static org.apache.ignite.internal.processors.metric.GridMetricManager.REBALANCED;
+import static org.apache.ignite.internal.processors.tracing.SpanType.EXCHANGE_FUTURE;
 
 /**
  * Partition exchange manager.
@@ -668,6 +671,27 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             // Event callback - without this callback future will never complete.
             exchFut.onEvent(exchId, evt, cache);
 
+            Span span = cctx.kernalContext().tracing().create(EXCHANGE_FUTURE, evt.span());
+
+            if (exchId != null) {
+                GridDhtPartitionExchangeId exchIdf = exchId;
+
+                span.addTag(SpanTags.tag(SpanTags.EVENT_NODE, SpanTags.ID), () -> evt.eventNode().id().toString());
+                span.addTag(SpanTags.tag(SpanTags.EVENT_NODE, SpanTags.CONSISTENT_ID),
+                    () -> evt.eventNode().consistentId().toString());
+                span.addTag(SpanTags.tag(SpanTags.EVENT, SpanTags.TYPE), () -> String.valueOf(evt.type()));
+                span.addTag(SpanTags.tag(SpanTags.EXCHANGE, SpanTags.ID), () -> String.valueOf(exchIdf.toString()));
+                span.addTag(SpanTags.tag(SpanTags.INITIAL, SpanTags.TOPOLOGY_VERSION, SpanTags.MAJOR),
+                    () -> String.valueOf(exchIdf.topologyVersion().topologyVersion()));
+                span.addTag(SpanTags.tag(SpanTags.INITIAL, SpanTags.TOPOLOGY_VERSION, SpanTags.MINOR),
+                    () -> String.valueOf(exchIdf.topologyVersion().minorTopologyVersion()));
+            }
+
+            span.addTag(SpanTags.NODE_ID, () -> cctx.localNodeId().toString());
+            span.addLog(() -> "Created");
+
+            exchFut.span(span);
+
             // Start exchange process.
             addFuture(exchFut);
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 97d46a1..31e88fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -577,7 +577,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         locCfgMgr = new GridLocalConfigManager(this, ctx);
 
-        transactions = new IgniteTransactionsImpl(sharedCtx, null);
+        transactions = new IgniteTransactionsImpl(sharedCtx, null, false);
 
         // Start shared managers.
         for (GridCacheSharedManager mgr : sharedCtx.managers())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 2432c50..06e554c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -64,6 +64,8 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -82,6 +84,8 @@ import org.jetbrains.annotations.Nullable;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRELOAD;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_DHT_LOCK_MAP;
 
 /**
  * Cache lock future.
@@ -91,6 +95,9 @@ public final class GridDhtLockFuture extends GridCacheCompoundIdentityFuture<Boo
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Tracing span. */
+    private Span span;
+
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
@@ -729,30 +736,32 @@ public final class GridDhtLockFuture extends GridCacheCompoundIdentityFuture<Boo
 
     /** {@inheritDoc} */
     @Override public boolean onDone(@Nullable Boolean success, @Nullable Throwable err) {
-        // Protect against NPE.
-        if (success == null) {
-            assert err != null;
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            // Protect against NPE.
+            if (success == null) {
+                assert err != null;
 
-            success = false;
-        }
+                success = false;
+            }
 
-        assert err == null || !success;
-        assert !success || (initialized() && !hasPending()) : "Invalid done callback [success=" + success +
-            ", fut=" + this + ']';
+            assert err == null || !success;
+            assert !success || (initialized() && !hasPending()) : "Invalid done callback [success=" + success +
+                ", fut=" + this + ']';
 
-        if (log.isDebugEnabled())
-            log.debug("Received onDone(..) callback [success=" + success + ", err=" + err + ", fut=" + this + ']');
+            if (log.isDebugEnabled())
+                log.debug("Received onDone(..) callback [success=" + success + ", err=" + err + ", fut=" + this + ']');
 
-        // If locks were not acquired yet, delay completion.
-        if (isDone() || (err == null && success && !checkLocks()))
-            return false;
+            // If locks were not acquired yet, delay completion.
+            if (isDone() || (err == null && success && !checkLocks()))
+                return false;
 
-        synchronized (this) {
-            if (this.err == null)
-                this.err = err;
-        }
+            synchronized (this) {
+                if (this.err == null)
+                    this.err = err;
+            }
 
-        return onComplete(success, err instanceof NodeStoppingException, true);
+            return onComplete(success, err instanceof NodeStoppingException, true);
+        }
     }
 
     /**
@@ -810,18 +819,21 @@ public final class GridDhtLockFuture extends GridCacheCompoundIdentityFuture<Boo
      *
      */
     public void map() {
-        if (F.isEmpty(entries)) {
-            onComplete(true, false, true);
+        try (TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_DHT_LOCK_MAP, MTC.span()))) {
+            if (F.isEmpty(entries)) {
+                onComplete(true, false, true);
 
-            return;
-        }
+                return;
+            }
 
-        readyLocks();
+            readyLocks();
 
-        if (timeout > 0 && !isDone()) { // Prevent memory leak if future is completed by call to readyLocks.
-            timeoutObj = new LockTimeoutObject();
+            if (timeout > 0 && !isDone()) { // Prevent memory leak if future is completed by call to readyLocks.
+                timeoutObj = new LockTimeoutObject();
 
-            cctx.time().addTimeoutObject(timeoutObj);
+                cctx.time().addTimeoutObject(timeoutObj);
+            }
         }
     }
 
@@ -898,7 +910,8 @@ public final class GridDhtLockFuture extends GridCacheCompoundIdentityFuture<Boo
                 return;
 
             if (log.isDebugEnabled())
-                log.debug("Mapped DHT lock future [dhtMap=" + F.nodeIds(dhtMap.keySet()) + ", dhtLockFut=" + this + ']');
+                log.debug("Mapped DHT lock future [dhtMap=" + F.nodeIds(dhtMap.keySet()) +
+                    ", dhtLockFut=" + this + ']');
 
             long timeout = inTx() ? tx.remainingTime() : this.timeout;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 4152993..dbed457 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -40,6 +40,8 @@ import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.TxCounters;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxSerializationCheckedException;
@@ -57,6 +59,8 @@ import org.apache.ignite.lang.IgniteUuid;
 import static java.util.Objects.isNull;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_DHT_FINISH;
 import static org.apache.ignite.transactions.TransactionState.COMMITTING;
 
 /**
@@ -67,6 +71,9 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Tracing span. */
+    private Span span;
+
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
@@ -228,58 +235,60 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
 
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx tx, Throwable err) {
-        if (initialized() || err != null) {
-            Throwable e = this.err;
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (initialized() || err != null) {
+                Throwable e = this.err;
 
-            if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) {
-                try {
-                    boolean nodeStopping = X.hasCause(err, NodeStoppingException.class);
+                if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) {
+                    try {
+                        boolean nodeStopping = X.hasCause(err, NodeStoppingException.class);
 
-                    this.tx.tmFinish(err == null, nodeStopping || cctx.kernalContext().failure().nodeStopping(), false);
-                }
-                catch (IgniteCheckedException finishErr) {
-                    U.error(log, "Failed to finish tx: " + tx, e);
+                        this.tx.tmFinish(err == null, nodeStopping || cctx.kernalContext().failure().nodeStopping(), false);
+                    }
+                    catch (IgniteCheckedException finishErr) {
+                        U.error(log, "Failed to finish tx: " + tx, e);
 
-                    if (e == null)
-                        e = finishErr;
+                        if (e == null)
+                            e = finishErr;
+                    }
                 }
-            }
 
-            if (commit && e == null)
-                e = this.tx.commitError();
+                if (commit && e == null)
+                    e = this.tx.commitError();
 
-            Throwable finishErr = e != null ? e : err;
+                Throwable finishErr = e != null ? e : err;
 
-            if (super.onDone(tx, finishErr)) {
-                cctx.tm().mvccFinish(this.tx);
+                if (super.onDone(tx, finishErr)) {
+                    cctx.tm().mvccFinish(this.tx);
 
-                if (finishErr == null)
-                    finishErr = this.tx.commitError();
+                    if (finishErr == null)
+                        finishErr = this.tx.commitError();
 
-                if (this.tx.syncMode() != PRIMARY_SYNC)
-                    this.tx.sendFinishReply(finishErr);
+                    if (this.tx.syncMode() != PRIMARY_SYNC)
+                        this.tx.sendFinishReply(finishErr);
 
-                if (!this.tx.txState().mvccEnabled() && !commit && shouldApplyCountersOnRollbackError(finishErr)) {
-                    TxCounters txCounters = this.tx.txCounters(false);
+                    if (!this.tx.txState().mvccEnabled() && !commit && shouldApplyCountersOnRollbackError(finishErr)) {
+                        TxCounters txCounters = this.tx.txCounters(false);
 
-                    if (txCounters != null) {
-                        try {
-                            cctx.tm().txHandler().applyPartitionsUpdatesCounters(txCounters.updateCounters(), true, true);
-                        }
-                        catch (IgniteCheckedException e0) {
-                            throw new IgniteException(e0);
+                        if (txCounters != null) {
+                            try {
+                                cctx.tm().txHandler().applyPartitionsUpdatesCounters(txCounters.updateCounters(), true, true);
+                            }
+                            catch (IgniteCheckedException e0) {
+                                throw new IgniteException(e0);
+                            }
                         }
                     }
-                }
 
-                // Don't forget to clean up.
-                cctx.mvcc().removeFuture(futId);
+                    // Don't forget to clean up.
+                    cctx.mvcc().removeFuture(futId);
 
-                return true;
+                    return true;
+                }
             }
-        }
 
-        return false;
+            return false;
+        }
     }
 
     /**
@@ -317,22 +326,25 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
      */
     @SuppressWarnings({"SimplifiableIfStatement"})
     public void finish(boolean commit) {
-        boolean sync;
+        try (MTC.TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_DHT_FINISH, MTC.span()))) {
+            boolean sync;
 
-        assert !tx.txState().mvccEnabled() || tx.mvccSnapshot() != null;
+            assert !tx.txState().mvccEnabled() || tx.mvccSnapshot() != null;
 
-        if (!F.isEmpty(dhtMap) || !F.isEmpty(nearMap))
-            sync = finish(commit, dhtMap, nearMap);
-        else if (!commit && !F.isEmpty(tx.lockTransactionNodes()))
-            sync = rollbackLockTransactions(tx.lockTransactionNodes());
-        else
-            // No backup or near nodes to send commit message to (just complete then).
-            sync = false;
+            if (!F.isEmpty(dhtMap) || !F.isEmpty(nearMap))
+                sync = finish(commit, dhtMap, nearMap);
+            else if (!commit && !F.isEmpty(tx.lockTransactionNodes()))
+                sync = rollbackLockTransactions(tx.lockTransactionNodes());
+            else
+                // No backup or near nodes to send commit message to (just complete then).
+                sync = false;
 
-        markInitialized();
+            markInitialized();
 
-        if (!sync)
-            onComplete();
+            if (!sync)
+                onComplete();
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 2902d61..6c30c54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -45,6 +45,8 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter;
 import org.apache.ignite.internal.processors.cache.transactions.TxCounters;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridLeanMap;
 import org.apache.ignite.internal.util.GridLeanSet;
@@ -118,6 +120,9 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         // No-op.
     }
 
+    /** Tracing span. */
+    private Span span = NoopSpan.INSTANCE;
+
     /**
      * @param xidVer Transaction version.
      * @param implicit Implicit flag.
@@ -936,6 +941,20 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         return prepFut;
     }
 
+    /**
+     * @return Tracing span.
+     */
+    public Span span() {
+        return span;
+    }
+
+    /**
+     * @param span New tracing span.
+     */
+    public void span(Span span) {
+        this.span = span;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(GridDhtTxLocalAdapter.class, this, "nearNodes", nearMap.keySet(),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 2ad04b4..c0acc2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -75,6 +75,8 @@ import org.apache.ignite.internal.processors.cache.transactions.TxCounters;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.GridLeanSet;
@@ -105,6 +107,8 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.NOO
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_DHT_PREPARE;
 import static org.apache.ignite.internal.util.lang.GridFunc.isEmpty;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
 
@@ -117,6 +121,9 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Tracing span. */
+    private Span span;
+
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
@@ -737,104 +744,106 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
 
     /** {@inheritDoc} */
     @Override public boolean onDone(GridNearTxPrepareResponse res0, Throwable err) {
-        assert err != null || (initialized() && !hasPending()) : "On done called for prepare future that has " +
-            "pending mini futures: " + this;
+        try (TraceSurroundings ignored2 = MTC.support(span)) {
+            assert err != null || (initialized() && !hasPending()) : "On done called for prepare future that has " +
+                "pending mini futures: " + this;
 
-        ERR_UPD.compareAndSet(this, null, err);
+            ERR_UPD.compareAndSet(this, null, err);
 
-        // Must clear prepare future before response is sent or listeners are notified.
-        if (tx.optimistic())
-            tx.clearPrepareFuture(this);
+            // Must clear prepare future before response is sent or listeners are notified.
+            if (tx.optimistic())
+                tx.clearPrepareFuture(this);
 
-        // Do not commit one-phase commit transaction if originating node has near cache enabled.
-        if (tx.commitOnPrepare()) {
-            assert last;
+            // Do not commit one-phase commit transaction if originating node has near cache enabled.
+            if (tx.commitOnPrepare()) {
+                assert last;
 
-            Throwable prepErr = this.err;
+                Throwable prepErr = this.err;
 
-            // Must create prepare response before transaction is committed to grab correct return value.
-            final GridNearTxPrepareResponse res = createPrepareResponse(prepErr);
+                // Must create prepare response before transaction is committed to grab correct return value.
+                final GridNearTxPrepareResponse res = createPrepareResponse(prepErr);
 
-            onComplete(res);
+                onComplete(res);
 
-            if (tx.markFinalizing(IgniteInternalTx.FinalizationStatus.USER_FINISH)) {
-                CIX1<IgniteInternalFuture<IgniteInternalTx>> resClo =
-                    new CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
-                        @Override public void applyx(IgniteInternalFuture<IgniteInternalTx> fut) {
-                            if (res.error() == null && fut.error() != null)
-                                res.error(fut.error());
+                if (tx.markFinalizing(IgniteInternalTx.FinalizationStatus.USER_FINISH)) {
+                    CIX1<IgniteInternalFuture<IgniteInternalTx>> resClo =
+                        new CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
+                            @Override public void applyx(IgniteInternalFuture<IgniteInternalTx> fut) {
+                                if (res.error() == null && fut.error() != null)
+                                    res.error(fut.error());
 
-                            if (REPLIED_UPD.compareAndSet(GridDhtTxPrepareFuture.this, 0, 1))
-                                sendPrepareResponse(res);
-                        }
-                    };
+                                if (REPLIED_UPD.compareAndSet(GridDhtTxPrepareFuture.this, 0, 1))
+                                    sendPrepareResponse(res);
+                            }
+                        };
 
-                try {
-                    if (prepErr == null) {
-                        try {
-                            tx.commitAsync().listen(resClo);
-                        }
-                        catch (Throwable e) {
-                            res.error(e);
+                    try {
+                        if (prepErr == null) {
+                            try {
+                                tx.commitAsync().listen(resClo);
+                            }
+                            catch (Throwable e) {
+                                res.error(e);
+
+                                tx.systemInvalidate(true);
 
-                            tx.systemInvalidate(true);
+                                try {
+                                    tx.rollbackAsync().listen(resClo);
+                                }
+                                catch (Throwable e1) {
+                                    e.addSuppressed(e1);
+                                }
 
+                                throw e;
+                            }
+                        }
+                        else if (!cctx.kernalContext().isStopping()) {
                             try {
                                 tx.rollbackAsync().listen(resClo);
                             }
-                            catch (Throwable e1) {
-                                e.addSuppressed(e1);
-                            }
+                            catch (Throwable e) {
+                                if (err != null)
+                                    err.addSuppressed(e);
 
-                            throw e;
+                                throw err;
+                            }
                         }
                     }
-                    else if (!cctx.kernalContext().isStopping()) {
-                        try {
-                            tx.rollbackAsync().listen(resClo);
-                        }
-                        catch (Throwable e) {
-                            if (err != null)
-                                err.addSuppressed(e);
+                    catch (Throwable e) {
+                        tx.logTxFinishErrorSafe(log, true, e);
 
-                            throw err;
-                        }
+                        cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
                     }
                 }
-                catch (Throwable e) {
-                    tx.logTxFinishErrorSafe(log, true, e);
 
-                    cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
-                }
+                return true;
             }
+            else {
+                if (REPLIED_UPD.compareAndSet(this, 0, 1)) {
+                    GridNearTxPrepareResponse res = createPrepareResponse(this.err);
 
-            return true;
-        }
-        else {
-            if (REPLIED_UPD.compareAndSet(this, 0, 1)) {
-                GridNearTxPrepareResponse res = createPrepareResponse(this.err);
-
-                // Will call super.onDone().
-                onComplete(res);
+                    // Will call super.onDone().
+                    onComplete(res);
 
-                sendPrepareResponse(res);
+                    sendPrepareResponse(res);
 
-                return true;
-            }
-            else {
-                // Other thread is completing future. Wait for it to complete.
-                try {
-                    if (err != null)
-                        get();
-                }
-                catch (IgniteInterruptedException e) {
-                    onError(new IgniteCheckedException("Got interrupted while waiting for replies to be sent.", e));
-                }
-                catch (IgniteCheckedException ignored) {
-                    // No-op, get() was just synchronization.
+                    return true;
                 }
+                else {
+                    // Other thread is completing future. Wait for it to complete.
+                    try {
+                        if (err != null)
+                            get();
+                    }
+                    catch (IgniteInterruptedException e) {
+                        onError(new IgniteCheckedException("Got interrupted while waiting for replies to be sent.", e));
+                    }
+                    catch (IgniteCheckedException ignored) {
+                        // No-op, get() was just synchronization.
+                    }
 
-                return false;
+                    return false;
+                }
             }
         }
     }
@@ -1044,18 +1053,19 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
      */
     public void prepare(GridNearTxPrepareRequest req) {
         assert req != null;
+        try (MTC.TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_DHT_PREPARE, MTC.span()))) {
+            if (tx.empty() && !req.queryUpdate()) {
+                tx.setRollbackOnly();
 
-        if (tx.empty() && !req.queryUpdate()) {
-            tx.setRollbackOnly();
-
-            onDone((GridNearTxPrepareResponse)null);
-        }
+                onDone((GridNearTxPrepareResponse)null);
+            }
 
-        this.req = req;
+            this.req = req;
 
-        ClusterNode node = cctx.discovery().node(tx.topologyVersion(), tx.nearNodeId());
+            ClusterNode node = cctx.discovery().node(tx.topologyVersion(), tx.nearNodeId());
 
-        boolean validateCache = needCacheValidation(node);
+            boolean validateCache = needCacheValidation(node);
 
         boolean writesEmpty = isEmpty(req.writes());
 
@@ -1070,35 +1080,36 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
             }
         }
 
-        boolean ser = tx.serializable() && tx.optimistic();
+            boolean ser = tx.serializable() && tx.optimistic();
 
         if (!writesEmpty || (ser && !F.isEmpty(req.reads()))) {
             Map<Integer, Collection<KeyCacheObject>> forceKeys = null;
 
-            for (IgniteTxEntry entry : req.writes())
-                forceKeys = checkNeedRebalanceKeys(entry, forceKeys);
-
-            if (ser) {
-                for (IgniteTxEntry entry : req.reads())
+                for (IgniteTxEntry entry : req.writes())
                     forceKeys = checkNeedRebalanceKeys(entry, forceKeys);
+
+                if (ser) {
+                    for (IgniteTxEntry entry : req.reads())
+                        forceKeys = checkNeedRebalanceKeys(entry, forceKeys);
+                }
+
+                forceKeysFut = forceRebalanceKeys(forceKeys);
             }
 
-            forceKeysFut = forceRebalanceKeys(forceKeys);
-        }
+            readyLocks();
 
-        readyLocks();
+            // Start timeout tracking after 'readyLocks' to avoid race with timeout processing.
+            if (timeoutObj != null) {
+                cctx.time().addTimeoutObject(timeoutObj);
 
-        // Start timeout tracking after 'readyLocks' to avoid race with timeout processing.
-        if (timeoutObj != null) {
-            cctx.time().addTimeoutObject(timeoutObj);
+                // Fix race with add/remove timeout object if locks are mapped from another
+                // thread before timeout object is enqueued.
+                if (tx.onePhaseCommit())
+                    timeoutAddedLatch.countDown();
+            }
 
-            // Fix race with add/remove timeout object if locks are mapped from another
-            // thread before timeout object is enqueued.
-            if (tx.onePhaseCommit())
-                timeoutAddedLatch.countDown();
+            mapIfLocked();
         }
-
-        mapIfLocked();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 43fb30a..8b6ebed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -62,6 +62,9 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanType;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -82,6 +85,8 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_COLOCATED_LOCK_MAP;
 
 /**
  * Colocated cache lock future.
@@ -91,6 +96,9 @@ public final class GridDhtColocatedLockFuture extends GridCacheCompoundIdentityF
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Tracing span. */
+    private Span span;
+
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
@@ -598,23 +606,25 @@ public final class GridDhtColocatedLockFuture extends GridCacheCompoundIdentityF
 
     /** {@inheritDoc} */
     @Override public boolean onDone(Boolean success, Throwable err) {
-        if (log.isDebugEnabled())
-            log.debug("Received onDone(..) callback [success=" + success + ", err=" + err + ", fut=" + this + ']');
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (log.isDebugEnabled())
+                log.debug("Received onDone(..) callback [success=" + success + ", err=" + err + ", fut=" + this + ']');
 
-        // Local GridDhtLockFuture
-        if (inTx() && this.err instanceof IgniteTxTimeoutCheckedException && cctx.tm().deadlockDetectionEnabled())
-            return false;
+            // Local GridDhtLockFuture
+            if (inTx() && this.err instanceof IgniteTxTimeoutCheckedException && cctx.tm().deadlockDetectionEnabled())
+                return false;
 
-        if (isDone())
-            return false;
+            if (isDone())
+                return false;
 
-        if (err != null)
-            onError(err);
+            if (err != null)
+                onError(err);
 
-        if (err != null)
-            success = false;
+            if (err != null)
+                success = false;
 
-        return onComplete(success, true);
+            return onComplete(success, true);
+        }
     }
 
     /**
@@ -752,75 +762,80 @@ public final class GridDhtColocatedLockFuture extends GridCacheCompoundIdentityF
      * part. Note that if primary node leaves grid, the future will fail and transaction will be rolled back.
      */
     void map() {
-        if (isDone()) // Possible due to async rollback.
-            return;
+        try (TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_COLOCATED_LOCK_MAP, MTC.span()))) {
+            if (isDone()) // Possible due to async rollback.
+                return;
 
-        if (timeout > 0) {
-            timeoutObj = new LockTimeoutObject();
+            if (timeout > 0) {
+                timeoutObj = new LockTimeoutObject();
 
-            cctx.time().addTimeoutObject(timeoutObj);
-        }
+                cctx.time().addTimeoutObject(timeoutObj);
+            }
 
-        // Obtain the topology version to use.
-        AffinityTopologyVersion topVer = cctx.mvcc().lastExplicitLockTopologyVersion(threadId);
+            // Obtain the topology version to use.
+            AffinityTopologyVersion topVer = cctx.mvcc().lastExplicitLockTopologyVersion(threadId);
 
-        // If there is another system transaction in progress, use it's topology version to prevent deadlock.
-        if (topVer == null && tx != null && tx.system())
-            topVer = cctx.tm().lockedTopologyVersion(Thread.currentThread().getId(), tx);
+            // If there is another system transaction in progress, use it's topology version to prevent deadlock.
+            if (topVer == null && tx != null && tx.system())
+                topVer = cctx.tm().lockedTopologyVersion(Thread.currentThread().getId(), tx);
 
-        if (topVer != null && tx != null)
-            tx.topologyVersion(topVer);
+            if (topVer != null && tx != null)
+                tx.topologyVersion(topVer);
 
-        if (topVer == null && tx != null)
-            topVer = tx.topologyVersionSnapshot();
+            if (topVer == null && tx != null)
+                topVer = tx.topologyVersionSnapshot();
 
-        if (topVer != null) {
-            AffinityTopologyVersion lastChangeVer = cctx.shared().exchange().lastAffinityChangedTopologyVersion(topVer);
+            if (topVer != null) {
+                AffinityTopologyVersion lastChangeVer =
+                    cctx.shared().exchange().lastAffinityChangedTopologyVersion(topVer);
 
-            IgniteInternalFuture<AffinityTopologyVersion> affFut = cctx.shared().exchange().affinityReadyFuture(lastChangeVer);
+                IgniteInternalFuture<AffinityTopologyVersion> affFut =
+                    cctx.shared().exchange().affinityReadyFuture(lastChangeVer);
 
-            if (!affFut.isDone()) {
-                try {
-                    affFut.get();
-                }
-                catch (IgniteCheckedException e) {
-                    onDone(err);
+                if (!affFut.isDone()) {
+                    try {
+                        affFut.get();
+                    }
+                    catch (IgniteCheckedException e) {
+                        onDone(err);
 
-                    return;
+                        return;
+                    }
                 }
-            }
 
-            for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()) {
-                if (fut.exchangeDone() && fut.topologyVersion().equals(lastChangeVer)) {
-                    Throwable err = fut.validateCache(cctx, recovery, read, null, keys);
+                for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()) {
+                    if (fut.exchangeDone() && fut.topologyVersion().equals(lastChangeVer)) {
+                        Throwable err = fut.validateCache(cctx, recovery, read, null, keys);
 
-                    if (err != null) {
-                        onDone(err);
+                        if (err != null) {
+                            onDone(err);
 
-                        return;
+                            return;
+                        }
+
+                        break;
                     }
+                }
 
-                    break;
+                // Continue mapping on the same topology version as it was before.
+                synchronized (this) {
+                    if (this.topVer == null)
+                        this.topVer = topVer;
                 }
-            }
 
-            // Continue mapping on the same topology version as it was before.
-            synchronized (this) {
-                if (this.topVer == null)
-                    this.topVer = topVer;
-            }
+                cctx.mvcc().addFuture(this);
 
-            cctx.mvcc().addFuture(this);
+                map(keys, false, true);
 
-            map(keys, false, true);
+                markInitialized();
 
-            markInitialized();
+                return;
+            }
 
-            return;
+            // Must get topology snapshot and map on that version.
+            mapOnTopology(false, null);
         }
-
-        // Must get topology snapshot and map on that version.
-        mapOnTopology(false, null);
     }
 
     /**
@@ -1172,14 +1187,17 @@ public final class GridDhtColocatedLockFuture extends GridCacheCompoundIdentityF
      * @throws IgniteCheckedException If failed.
      */
     private void proceedMapping() throws IgniteCheckedException {
-        boolean set = tx != null && cctx.shared().tm().setTxTopologyHint(tx.topologyVersionSnapshot());
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(SpanType.TX_MAP_PROCEED, MTC.span()))) {
+            boolean set = tx != null && cctx.shared().tm().setTxTopologyHint(tx.topologyVersionSnapshot());
 
-        try {
-            proceedMapping0();
-        }
-        finally {
-            if (set)
-                cctx.tm().setTxTopologyHint(null);
+            try {
+                proceedMapping0();
+            }
+            finally {
+                if (set)
+                    cctx.tm().setTxTopologyHint(null);
+            }
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 21b3d26..5788820 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -112,6 +112,9 @@ import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.service.GridServiceProcessor;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanTags;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.TimeBag;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -388,6 +391,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     /** Some of owned by affinity partitions were changed state to moving on this exchange. */
     private volatile boolean affinityReassign;
 
+    /** Tracing span. */
+    private Span span = NoopSpan.INSTANCE;
+
     /**
      * @param cctx Cache context.
      * @param busyLock Busy lock.
@@ -432,6 +438,24 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
+     * Set span.
+     *
+     * @param span Span.
+     */
+    public void span(Span span) {
+        this.span = span;
+    }
+
+    /**
+     * Gets span instance.
+     *
+     * @return Span.
+     */
+    public Span span() {
+        return span;
+    }
+
+    /**
      * @return Future mutex.
      */
     public Object mutex() {
@@ -874,6 +898,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     ", exchangeFreeSwitch=" + exchCtx.exchangeFreeSwitch() + ']');
             }
 
+            span.addLog(() -> "Exchange parameters initialization");
+
             timeBag.finishGlobalStage("Exchange parameters initialization");
 
             ExchangeType exchange;
@@ -2412,6 +2438,19 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         assert res != null || err != null;
 
+        if (res != null) {
+            span.addTag(SpanTags.tag(SpanTags.RESULT, SpanTags.TOPOLOGY_VERSION, SpanTags.MAJOR),
+                () -> String.valueOf(res.topologyVersion()));
+            span.addTag(SpanTags.tag(SpanTags.RESULT, SpanTags.TOPOLOGY_VERSION, SpanTags.MINOR),
+                () -> String.valueOf(res.minorTopologyVersion()));
+        }
+
+        if (err != null) {
+            Throwable errf = err;
+
+            span.addTag(SpanTags.ERROR, errf::toString);
+        }
+
         try {
             waitUntilNewCachesAreRegistered();
 
@@ -2552,6 +2591,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (super.onDone(res, err)) {
             afterLsnrCompleteFut.onDone();
 
+            span.addLog(() -> "Completed partition exchange");
+
+            span.end();
+
             if (err == null) {
                 updateDurationHistogram(System.currentTimeMillis() - initTime);
 
@@ -3688,6 +3731,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         try {
             initFut.get();
 
+            span.addLog(() -> "Waiting for all single messages");
+
             timeBag.finishGlobalStage("Waiting for all single messages");
 
             assert crd.isLocal();
@@ -3804,6 +3849,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     });
             }
 
+            span.addLog(() -> "Affinity recalculation (crd)");
+
             timeBag.finishGlobalStage("Affinity recalculation (crd)");
 
             Map<Integer, CacheGroupAffinityMessage> joinedNodeAff = new ConcurrentHashMap<>(cctx.cache().cacheGroups().size());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index 1a7cfb0..82672e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapp
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -60,6 +61,7 @@ import org.apache.ignite.lang.IgniteReducer;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
 import static org.apache.ignite.transactions.TransactionState.PREPARING;
 
@@ -170,22 +172,24 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
      * @param e Error.
      */
     private void onError(@Nullable GridDistributedTxMapping m, Throwable e) {
-        if (X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, ClusterTopologyException.class)) {
-            if (tx.onePhaseCommit()) {
-                tx.markForBackupCheck();
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, ClusterTopologyException.class)) {
+                if (tx.onePhaseCommit()) {
+                    tx.markForBackupCheck();
 
-                onComplete();
+                    onComplete();
 
-                return;
+                    return;
+                }
             }
-        }
 
-        if (e instanceof IgniteTxOptimisticCheckedException) {
-            if (m != null)
-                tx.removeMapping(m.primary().id());
-        }
+            if (e instanceof IgniteTxOptimisticCheckedException) {
+                if (m != null)
+                    tx.removeMapping(m.primary().id());
+            }
 
-        prepareError(e);
+            prepareError(e);
+        }
     }
 
     /**
@@ -210,17 +214,19 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
 
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx t, Throwable err) {
-        if (isDone())
-            return false;
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (isDone())
+                return false;
 
-        if (err != null) {
-            ERR_UPD.compareAndSet(this, null, err);
+            if (err != null) {
+                ERR_UPD.compareAndSet(this, null, err);
 
-            if (keyLockFut != null)
-                keyLockFut.onDone(err);
-        }
+                if (keyLockFut != null)
+                    keyLockFut.onDone(err);
+            }
 
-        return onComplete();
+            return onComplete();
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 71d436f..0954974 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
@@ -67,6 +68,8 @@ import org.apache.ignite.transactions.TransactionTimeoutException;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.MTC.support;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
 import static org.apache.ignite.transactions.TransactionState.PREPARING;
 
@@ -139,24 +142,26 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
      * @param discoThread {@code True} if executed from discovery thread.
      */
     private void onError(Throwable e, boolean discoThread) {
-        if (e instanceof IgniteTxTimeoutCheckedException) {
-            onTimeout();
+        try (TraceSurroundings ignored = support(span)) {
+            if (e instanceof IgniteTxTimeoutCheckedException) {
+                onTimeout();
 
-            return;
-        }
+                return;
+            }
 
-        if (X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, ClusterTopologyException.class)) {
-            if (tx.onePhaseCommit()) {
-                tx.markForBackupCheck();
+            if (X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, ClusterTopologyException.class)) {
+                if (tx.onePhaseCommit()) {
+                    tx.markForBackupCheck();
 
-                onComplete();
+                    onComplete();
 
-                return;
+                    return;
+                }
             }
-        }
 
-        if (ERR_UPD.compareAndSet(this, null, e))
-            onComplete();
+            if (ERR_UPD.compareAndSet(this, null, e))
+                onComplete();
+        }
     }
 
     /** {@inheritDoc} */
@@ -250,12 +255,14 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx t, Throwable err) {
-        if (isDone())
-            return false;
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (isDone())
+                return false;
 
-        ERR_UPD.compareAndSet(this, null, err);
+            ERR_UPD.compareAndSet(this, null, err);
 
-        return onComplete();
+            return onComplete();
+        }
     }
 
     /**
@@ -704,59 +711,61 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
      *
      */
     private void onTimeout() {
-        if (cctx.tm().deadlockDetectionEnabled()) {
-            Set<IgniteTxKey> keys = null;
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (cctx.tm().deadlockDetectionEnabled()) {
+                Set<IgniteTxKey> keys = null;
 
-            if (keyLockFut != null)
-                keys = new HashSet<>(keyLockFut.lockKeys);
-            else {
-                synchronized (this) {
-                    int size = futuresCountNoLock();
+                if (keyLockFut != null)
+                    keys = new HashSet<>(keyLockFut.lockKeys);
+                else {
+                    synchronized (this) {
+                        int size = futuresCountNoLock();
 
-                    for (int i = 0; i < size; i++) {
-                        IgniteInternalFuture fut = future(i);
+                        for (int i = 0; i < size; i++) {
+                            IgniteInternalFuture fut = future(i);
 
-                        if (isMini(fut) && !fut.isDone()) {
-                            MiniFuture miniFut = (MiniFuture)fut;
+                            if (isMini(fut) && !fut.isDone()) {
+                                MiniFuture miniFut = (MiniFuture)fut;
 
-                            Collection<IgniteTxEntry> entries = miniFut.mapping().entries();
+                                Collection<IgniteTxEntry> entries = miniFut.mapping().entries();
 
-                            keys = U.newHashSet(entries.size());
+                                keys = U.newHashSet(entries.size());
 
-                            for (IgniteTxEntry entry : entries)
-                                keys.add(entry.txKey());
+                                for (IgniteTxEntry entry : entries)
+                                    keys.add(entry.txKey());
 
-                            break;
+                                break;
+                            }
                         }
                     }
                 }
-            }
 
-            add(new GridEmbeddedFuture<>(new IgniteBiClosure<TxDeadlock, Exception, Object>() {
-                @Override public GridNearTxPrepareResponse apply(TxDeadlock deadlock, Exception e) {
-                    if (e != null)
-                        U.warn(log, "Failed to detect deadlock.", e);
-                    else {
-                        e = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " +
-                            "transaction [timeout=" + tx.timeout() + ", tx=" + CU.txString(tx) + ']',
-                            deadlock != null ? new TransactionDeadlockException(deadlock.toString(cctx)) : null);
+                add(new GridEmbeddedFuture<>(new IgniteBiClosure<TxDeadlock, Exception, Object>() {
+                    @Override public GridNearTxPrepareResponse apply(TxDeadlock deadlock, Exception e) {
+                        if (e != null)
+                            U.warn(log, "Failed to detect deadlock.", e);
+                        else {
+                            e = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " +
+                                "transaction [timeout=" + tx.timeout() + ", tx=" + CU.txString(tx) + ']',
+                                deadlock != null ? new TransactionDeadlockException(deadlock.toString(cctx)) : null);
 
-                        if (!ERR_UPD.compareAndSet(GridNearOptimisticTxPrepareFuture.this, null, e) && err instanceof IgniteTxTimeoutCheckedException) {
-                            err = e;
+                            if (!ERR_UPD.compareAndSet(GridNearOptimisticTxPrepareFuture.this, null, e) && err instanceof IgniteTxTimeoutCheckedException) {
+                                err = e;
+                            }
                         }
-                    }
 
-                    onDone(null, e);
+                        onDone(null, e);
 
-                    return null;
-                }
-            }, cctx.tm().detectDeadlock(tx, keys)));
-        }
-        else {
-            ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException("Failed to acquire lock " +
-                "within provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
+                        return null;
+                    }
+                }, cctx.tm().detectDeadlock(tx, keys)));
+            }
+            else {
+                ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException("Failed to acquire lock " +
+                    "within provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
 
-            onComplete();
+                onComplete();
+            }
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
index eee6612..c0fbfc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
@@ -25,6 +25,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
@@ -35,6 +37,9 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_PREPARE;
+
 /**
  *
  */
@@ -42,6 +47,9 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
     /** */
     private static final long serialVersionUID = 7460376140787916619L;
 
+    /** Tracing span. */
+    protected Span span;
+
     /** */
     @GridToStringExclude
     protected KeyLockFuture keyLockFut;
@@ -80,40 +88,46 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
 
     /** {@inheritDoc} */
     @Override public final void onNearTxLocalTimeout() {
-        if (keyLockFut != null && !keyLockFut.isDone()) {
-            ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException("Failed to acquire lock " +
-                    "within provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (keyLockFut != null && !keyLockFut.isDone()) {
+                ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException(
+                    "Failed to acquire lock within provided timeout for transaction [timeout=" + tx.timeout() +
+                        ", tx=" + tx + ']'));
 
-            keyLockFut.onDone();
+                keyLockFut.onDone();
+            }
         }
     }
 
     /** {@inheritDoc} */
     @Override public final void prepare() {
-        // Obtain the topology version to use.
-        long threadId = Thread.currentThread().getId();
+        try (TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_NEAR_PREPARE, MTC.span()))) {
+            // Obtain the topology version to use.
+            long threadId = Thread.currentThread().getId();
 
-        AffinityTopologyVersion topVer = cctx.mvcc().lastExplicitLockTopologyVersion(threadId);
+            AffinityTopologyVersion topVer = cctx.mvcc().lastExplicitLockTopologyVersion(threadId);
 
-        // If there is another system transaction in progress, use it's topology version to prevent deadlock.
-        if (topVer == null && tx.system()) {
-            topVer = cctx.tm().lockedTopologyVersion(threadId, tx);
+            // If there is another system transaction in progress, use it's topology version to prevent deadlock.
+            if (topVer == null && tx.system()) {
+                topVer = cctx.tm().lockedTopologyVersion(threadId, tx);
 
-            if (topVer == null)
-                topVer = tx.topologyVersionSnapshot();
-        }
+                if (topVer == null)
+                    topVer = tx.topologyVersionSnapshot();
+            }
 
-        if (topVer != null) {
-            tx.topologyVersion(topVer);
+            if (topVer != null) {
+                tx.topologyVersion(topVer);
 
-            cctx.mvcc().addFuture(this);
+                cctx.mvcc().addFuture(this);
 
-            prepare0(false, true);
+                prepare0(false, true);
 
-            return;
-        }
+                return;
+            }
 
-        prepareOnTopology(false, null);
+            prepareOnTopology(false, null);
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 4680e66..5300ad3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -38,6 +38,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapp
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.C1;
@@ -48,6 +50,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_PREPARE;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
 import static org.apache.ignite.transactions.TransactionState.PREPARING;
 
@@ -58,6 +62,9 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
     /** */
     private static final long serialVersionUID = 4014479758215810181L;
 
+    /** Tracing span. */
+    private Span span;
+
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -166,29 +173,32 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
 
     /** {@inheritDoc} */
     @Override public void prepare() {
-        if (!tx.state(PREPARING)) {
-            if (tx.isRollbackOnly() || tx.setRollbackOnly()) {
-                if (tx.remainingTime() == -1)
-                    onDone(tx.timeoutException());
+        try (TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_NEAR_PREPARE, MTC.span()))) {
+            if (!tx.state(PREPARING)) {
+                if (tx.isRollbackOnly() || tx.setRollbackOnly()) {
+                    if (tx.remainingTime() == -1)
+                        onDone(tx.timeoutException());
+                    else
+                        onDone(tx.rollbackException());
+                }
                 else
-                    onDone(tx.rollbackException());
-            }
-            else
-                onDone(new IgniteCheckedException("Invalid transaction state for prepare " +
-                    "[state=" + tx.state() + ", tx=" + this + ']'));
+                    onDone(new IgniteCheckedException("Invalid transaction state for prepare " +
+                        "[state=" + tx.state() + ", tx=" + this + ']'));
 
-            return;
-        }
+                return;
+            }
 
-        try {
-            tx.userPrepare(Collections.<IgniteTxEntry>emptyList());
+            try {
+                tx.userPrepare(Collections.<IgniteTxEntry>emptyList());
 
-            cctx.mvcc().addFuture(this);
+                cctx.mvcc().addFuture(this);
 
-            preparePessimistic();
-        }
-        catch (IgniteCheckedException e) {
-            onDone(e);
+                preparePessimistic();
+            }
+            catch (IgniteCheckedException e) {
+                onDone(e);
+            }
         }
     }
 
@@ -433,22 +443,24 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
 
     /** {@inheritDoc} */
     @Override public boolean onDone(@Nullable IgniteInternalTx res, @Nullable Throwable err) {
-        if (err != null)
-            ERR_UPD.compareAndSet(GridNearPessimisticTxPrepareFuture.this, null, err);
+        try (TraceSurroundings ignored = MTC.support(span)) {
+            if (err != null)
+                ERR_UPD.compareAndSet(GridNearPessimisticTxPrepareFuture.this, null, err);
 
-        err = this.err;
+            err = this.err;
 
-        if ((!tx.onePhaseCommit() || tx.mappings().get(cctx.localNodeId()) == null) &&
-            (err == null || tx.needCheckBackup()))
-            tx.state(PREPARED);
+            if ((!tx.onePhaseCommit() || tx.mappings().get(cctx.localNodeId()) == null) &&
+                (err == null || tx.needCheckBackup()))
+                tx.state(PREPARED);
 
-        if (super.onDone(tx, err)) {
-            cctx.mvcc().removeVersionedFuture(this);
+            if (super.onDone(tx, err)) {
+                cctx.mvcc().removeVersionedFuture(this);
 
-            return true;
-        }
+                return true;
+            }
 
-        return false;
+            return false;
+        }
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index cc1509b..fc239da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -43,6 +43,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFini
 import org.apache.ignite.internal.processors.cache.mvcc.MvccFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -58,6 +60,9 @@ import org.apache.ignite.transactions.TransactionRollbackException;
 
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.MTC.support;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_FINISH;
 import static org.apache.ignite.transactions.TransactionState.UNKNOWN;
 
 /**
@@ -68,6 +73,9 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Tracing span. */
+    private Span span;
+
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
@@ -288,73 +296,75 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
 
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx tx0, Throwable err) {
-        if (isDone())
-            return false;
-
-        synchronized (this) {
+        try (MTC.TraceSurroundings ignored = support(span)) {
             if (isDone())
                 return false;
 
-            boolean nodeStop = false;
+            synchronized (this) {
+                if (isDone())
+                    return false;
 
-            if (err != null) {
-                tx.setRollbackOnly();
+                boolean nodeStop = false;
 
-                nodeStop = err instanceof NodeStoppingException || cctx.kernalContext().failure().nodeStopping();
-            }
+                if (err != null) {
+                    tx.setRollbackOnly();
 
-            if (commit) {
-                if (tx.commitError() != null)
-                    err = tx.commitError();
-                else if (err != null)
-                    tx.commitError(err);
-            }
+                    nodeStop = err instanceof NodeStoppingException || cctx.kernalContext().failure().nodeStopping();
+                }
 
-            if (initialized() || err != null) {
-                if (tx.needCheckBackup()) {
-                    assert tx.onePhaseCommit();
+                if (commit) {
+                    if (tx.commitError() != null)
+                        err = tx.commitError();
+                    else if (err != null)
+                        tx.commitError(err);
+                }
 
-                    if (err != null)
-                        err = new TransactionRollbackException("Failed to commit transaction.", err);
+                if (initialized() || err != null) {
+                    if (tx.needCheckBackup()) {
+                        assert tx.onePhaseCommit();
 
-                    try {
-                        tx.localFinish(err == null, true);
-                    }
-                    catch (IgniteCheckedException e) {
                         if (err != null)
-                            err.addSuppressed(e);
-                        else
-                            err = e;
+                            err = new TransactionRollbackException("Failed to commit transaction.", err);
+
+                        try {
+                            tx.localFinish(err == null, true);
+                        }
+                        catch (IgniteCheckedException e) {
+                            if (err != null)
+                                err.addSuppressed(e);
+                            else
+                                err = e;
+                        }
                     }
-                }
 
-                if (tx.onePhaseCommit()) {
-                    boolean commit = this.commit && err == null;
+                    if (tx.onePhaseCommit()) {
+                        boolean commit = this.commit && err == null;
 
-                    if (!nodeStop)
-                        finishOnePhase(commit);
+                        if (!nodeStop)
+                            finishOnePhase(commit);
 
-                    try {
-                        tx.tmFinish(commit, nodeStop, true);
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to finish tx: " + tx, e);
+                        try {
+                            tx.tmFinish(commit, nodeStop, true);
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to finish tx: " + tx, e);
 
-                        if (err == null)
-                            err = e;
+                            if (err == null)
+                                err = e;
+                        }
                     }
-                }
 
-                if (super.onDone(tx0, err)) {
-                    // Don't forget to clean up.
-                    cctx.mvcc().removeFuture(futId);
+                    if (super.onDone(tx0, err)) {
+                        // Don't forget to clean up.
+                        cctx.mvcc().removeFuture(futId);
 
-                    return true;
+                        return true;
+                    }
                 }
             }
-        }
 
-        return false;
+            return false;
+        }
     }
 
     /**
@@ -369,25 +379,28 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
 
     /** {@inheritDoc} */
     @Override public void finish(final boolean commit, final boolean clearThreadMap, final boolean onTimeout) {
-        if (!cctx.mvcc().addFuture(this, futureId()))
-            return;
+        try (TraceSurroundings ignored =
+                 MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_NEAR_FINISH, MTC.span()))) {
+            if (!cctx.mvcc().addFuture(this, futureId()))
+                return;
 
-        if (tx.onNeedCheckBackup()) {
-            assert tx.onePhaseCommit();
+            if (tx.onNeedCheckBackup()) {
+                assert tx.onePhaseCommit();
 
-            checkBackup();
+                checkBackup();
 
-            // If checkBackup is set, it means that primary node has crashed and we will not need to send
-            // finish request to it, so we can mark future as initialized.
-            markInitialized();
+                // If checkBackup is set, it means that primary node has crashed and we will not need to send
+                // finish request to it, so we can mark future as initialized.
+                markInitialized();
 
-            return;
-        }
+                return;
+            }
 
-        if (!commit && !clearThreadMap)
-            rollbackAsyncSafe(onTimeout);
-        else
-            doFinish(commit, clearThreadMap);
+            if (!commit && !clearThreadMap)
+                rollbackAsyncSafe(onTimeout);
+            else
+                doFinish(commit, clearThreadMap);
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 5739028..3f828b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -86,6 +86,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.EnlistOperation;
 import org.apache.ignite.internal.processors.query.UpdateSourceIterator;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
@@ -127,6 +128,9 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRA
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.SER_READ_EMPTY_ENTRY_VER;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.SER_READ_NOT_EMPTY_VER;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_ENLIST_READ;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_ENLIST_WRITE;
 import static org.apache.ignite.transactions.TransactionState.ACTIVE;
 import static org.apache.ignite.transactions.TransactionState.COMMITTED;
 import static org.apache.ignite.transactions.TransactionState.COMMITTING;
@@ -273,6 +277,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
      * @param taskNameHash Task name hash code.
      * @param lb Label.
      * @param txDumpsThrottling Log throttling information.
+     * @param tracingEnabled {@code true} if the transaction should be traced.
      */
     public GridNearTxLocal(
         GridCacheSharedContext ctx,
@@ -289,7 +294,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
         @Nullable UUID subjId,
         int taskNameHash,
         @Nullable String lb,
-        IgniteTxManager.TxDumpsThrottling txDumpsThrottling
+        IgniteTxManager.TxDumpsThrottling txDumpsThrottling,
+        boolean tracingEnabled
     ) {
         super(
             ctx,
@@ -1070,88 +1076,91 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
         boolean keepBinary,
         boolean recovery,
         Byte dataCenterId) {
-        GridFutureAdapter<Void> enlistFut = new GridFutureAdapter<>();
+        try (TraceSurroundings ignored2 =
+                 MTC.support(context().kernalContext().tracing().create(TX_NEAR_ENLIST_WRITE, MTC.span()))) {
+            GridFutureAdapter<Void> enlistFut = new GridFutureAdapter<>();
 
-        try {
-            if (!updateLockFuture(null, enlistFut))
-                return finishFuture(enlistFut, timedOut() ? timeoutException() : rollbackException(), false);
-
-            addActiveCache(cacheCtx, recovery);
-
-            final boolean hasFilters = !F.isEmptyOrNulls(filter) && !F.isAlwaysTrue(filter);
-            final boolean needVal = singleRmv || retval || hasFilters;
-            final boolean needReadVer = needVal && (serializable() && optimistic());
-
-            if (entryProcessor != null)
-                transform = true;
+            try {
+                if (!updateLockFuture(null, enlistFut))
+                    return finishFuture(enlistFut, timedOut() ? timeoutException() : rollbackException(), false);
 
-            GridCacheVersion drVer = dataCenterId != null ? cctx.versions().next(dataCenterId) : null;
+                addActiveCache(cacheCtx, recovery);
 
-            boolean loadMissed = enlistWriteEntry(cacheCtx,
-                entryTopVer,
-                cacheKey,
-                val,
-                entryProcessor,
-                invokeArgs,
-                expiryPlc,
-                retval,
-                lockOnly,
-                filter,
-                /*drVer*/drVer,
-                /*drTtl*/-1L,
-                /*drExpireTime*/-1L,
-                ret,
-                /*enlisted*/null,
-                skipStore,
-                singleRmv,
-                hasFilters,
-                needVal,
-                needReadVer,
-                keepBinary,
-                recovery);
+                final boolean hasFilters = !F.isEmptyOrNulls(filter) && !F.isAlwaysTrue(filter);
+                final boolean needVal = singleRmv || retval || hasFilters;
+                final boolean needReadVer = needVal && (serializable() && optimistic());
 
-            if (loadMissed) {
-                AffinityTopologyVersion topVer = topologyVersionSnapshot();
+                if (entryProcessor != null)
+                    transform = true;
 
-                if (topVer == null)
-                    topVer = entryTopVer;
+                GridCacheVersion drVer = dataCenterId != null ? cctx.versions().next(dataCenterId) : null;
 
-                IgniteInternalFuture<Void> loadFut = loadMissing(cacheCtx,
-                    topVer != null ? topVer : topologyVersion(),
-                    Collections.singleton(cacheKey),
+                boolean loadMissed = enlistWriteEntry(cacheCtx,
+                    entryTopVer,
+                    cacheKey,
+                    val,
+                    entryProcessor,
+                    invokeArgs,
+                    expiryPlc,
+                    retval,
+                    lockOnly,
                     filter,
+                    /*drVer*/drVer,
+                    /*drTtl*/-1L,
+                    /*drExpireTime*/-1L,
                     ret,
-                    needReadVer,
+                    /*enlisted*/null,
+                    skipStore,
                     singleRmv,
                     hasFilters,
-                    /*read through*/(entryProcessor != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore,
-                    retval,
+                    needVal,
+                    needReadVer,
                     keepBinary,
-                    recovery,
-                    expiryPlc);
+                    recovery);
 
-                loadFut.listen(new IgniteInClosure<IgniteInternalFuture<Void>>() {
-                    @Override public void apply(IgniteInternalFuture<Void> fut) {
-                        try {
-                            fut.get();
+                if (loadMissed) {
+                    AffinityTopologyVersion topVer = topologyVersionSnapshot();
 
-                            finishFuture(enlistFut, null, true);
-                        }
-                        catch (IgniteCheckedException e) {
-                            finishFuture(enlistFut, e, true);
+                    if (topVer == null)
+                        topVer = entryTopVer;
+
+                    IgniteInternalFuture<Void> loadFut = loadMissing(cacheCtx,
+                        topVer != null ? topVer : topologyVersion(),
+                        Collections.singleton(cacheKey),
+                        filter,
+                        ret,
+                        needReadVer,
+                        singleRmv,
+                        hasFilters,
+                        /*read through*/(entryProcessor != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore,
+                        retval,
+                        keepBinary,
+                        recovery,
+                        expiryPlc);
+
+                    loadFut.listen(new IgniteInClosure<IgniteInternalFuture<Void>>() {
+                        @Override public void apply(IgniteInternalFuture<Void> fut) {
+                            try {
+                                fut.get();
+
+                                finishFuture(enlistFut, null, true);
+                            }
+                            catch (IgniteCheckedException e) {
+                                finishFuture(enlistFut, e, true);
+                            }
                         }
-                    }
-                });
+                    });
 
-                return enlistFut;
-            }
+                    return enlistFut;
+                }
 
-            finishFuture(enlistFut, null, true);
+                finishFuture(enlistFut, null, true);
 
-            return enlistFut;
-        }
-        catch (IgniteCheckedException e) {
-            return finishFuture(enlistFut, e, true);
+                return enlistFut;
+            }
+            catch (IgniteCheckedException e) {
+                return finishFuture(enlistFut, e, true);
+            }
         }
     }
 
@@ -1201,59 +1210,61 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
     ) {
         assert retval || invokeMap == null;
 
-        GridFutureAdapter<Void> enlistFut = new GridFutureAdapter<>();
+        try (TraceSurroundings ignored2 =
+                 MTC.support(context().kernalContext().tracing().create(TX_NEAR_ENLIST_WRITE, MTC.span()))) {
+            GridFutureAdapter<Void> enlistFut = new GridFutureAdapter<>();
 
-        if (!updateLockFuture(null, enlistFut))
-            return finishFuture(enlistFut, timedOut() ? timeoutException() : rollbackException(), false);
+            if (!updateLockFuture(null, enlistFut))
+                return finishFuture(enlistFut, timedOut() ? timeoutException() : rollbackException(), false);
 
-        try {
-            addActiveCache(cacheCtx, recovery);
-        }
-        catch (IgniteCheckedException e) {
-            return finishFuture(enlistFut, e, false);
-        }
+            try {
+                addActiveCache(cacheCtx, recovery);
+            }
+            catch (IgniteCheckedException e) {
+                return finishFuture(enlistFut, e, false);
+            }
 
-        boolean rmv = lookup == null && invokeMap == null;
+            boolean rmv = lookup == null && invokeMap == null;
 
-        final boolean hasFilters = !F.isEmptyOrNulls(filter) && !F.isAlwaysTrue(filter);
-        final boolean needVal = singleRmv || retval || hasFilters;
-        final boolean needReadVer = needVal && (serializable() && optimistic());
+            final boolean hasFilters = !F.isEmptyOrNulls(filter) && !F.isAlwaysTrue(filter);
+            final boolean needVal = singleRmv || retval || hasFilters;
+            final boolean needReadVer = needVal && (serializable() && optimistic());
 
-        try {
-            // Set transform flag for transaction.
-            if (invokeMap != null)
-                transform = true;
+            try {
+                // Set transform flag for transaction.
+                if (invokeMap != null)
+                    transform = true;
 
-            Set<KeyCacheObject> missedForLoad = null;
+                Set<KeyCacheObject> missedForLoad = null;
 
-            for (Object key : keys) {
-                if (isRollbackOnly())
-                    return finishFuture(enlistFut, timedOut() ? timeoutException() : rollbackException(), false);
+                for (Object key : keys) {
+                    if (isRollbackOnly())
+                        return finishFuture(enlistFut, timedOut() ? timeoutException() : rollbackException(), false);
 
-                if (key == null) {
-                    rollback();
+                    if (key == null) {
+                        rollback();
 
-                    throw new NullPointerException("Null key.");
-                }
+                        throw new NullPointerException("Null key.");
+                    }
 
-                Object val = rmv || lookup == null ? null : lookup.get(key);
-                EntryProcessor entryProcessor = invokeMap == null ? null : invokeMap.get(key);
+                    Object val = rmv || lookup == null ? null : lookup.get(key);
+                    EntryProcessor entryProcessor = invokeMap == null ? null : invokeMap.get(key);
 
-                GridCacheVersion drVer;
-                long drTtl;
-                long drExpireTime;
+                    GridCacheVersion drVer;
+                    long drTtl;
+                    long drExpireTime;
 
-                if (drPutMap != null) {
-                    GridCacheDrInfo info = drPutMap.get(key);
+                    if (drPutMap != null) {
+                        GridCacheDrInfo info = drPutMap.get(key);
 
-                    assert info != null;
+                        assert info != null;
 
-                    drVer = info.version();
-                    drTtl = info.ttl();
-                    drExpireTime = info.expireTime();
-                }
-                else if (drRmvMap != null) {
-                    assert drRmvMap.get(key) != null;
+                        drVer = info.version();
+                        drTtl = info.ttl();
+                        drExpireTime = info.expireTime();
+                    }
+                    else if (drRmvMap != null) {
+                        assert drRmvMap.get(key) != null;
 
                     drVer = drRmvMap.get(key);
                     drTtl = -1L;
@@ -1270,85 +1281,86 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
                     drExpireTime = -1L;
                 }
 
-                if (!rmv && val == null && entryProcessor == null) {
-                    setRollbackOnly();
+                    if (!rmv && val == null && entryProcessor == null) {
+                        setRollbackOnly();
 
-                    throw new NullPointerException("Null value.");
-                }
+                        throw new NullPointerException("Null value.");
+                    }
 
-                KeyCacheObject cacheKey = cacheCtx.toCacheKeyObject(key);
+                    KeyCacheObject cacheKey = cacheCtx.toCacheKeyObject(key);
 
-                boolean loadMissed = enlistWriteEntry(cacheCtx,
-                    entryTopVer,
-                    cacheKey,
-                    val,
-                    entryProcessor,
-                    invokeArgs,
-                    expiryPlc,
-                    retval,
-                    lockOnly,
-                    filter,
-                    drVer,
-                    drTtl,
-                    drExpireTime,
-                    ret,
-                    enlisted,
-                    skipStore,
-                    singleRmv,
-                    hasFilters,
-                    needVal,
-                    needReadVer,
-                    keepBinary,
-                    recovery);
+                    boolean loadMissed = enlistWriteEntry(cacheCtx,
+                        entryTopVer,
+                        cacheKey,
+                        val,
+                        entryProcessor,
+                        invokeArgs,
+                        expiryPlc,
+                        retval,
+                        lockOnly,
+                        filter,
+                        drVer,
+                        drTtl,
+                        drExpireTime,
+                        ret,
+                        enlisted,
+                        skipStore,
+                        singleRmv,
+                        hasFilters,
+                        needVal,
+                        needReadVer,
+                        keepBinary,
+                        recovery);
 
-                if (loadMissed) {
-                    if (missedForLoad == null)
-                        missedForLoad = new HashSet<>();
+                    if (loadMissed) {
+                        if (missedForLoad == null)
+                            missedForLoad = new HashSet<>();
 
-                    missedForLoad.add(cacheKey);
+                        missedForLoad.add(cacheKey);
+                    }
                 }
-            }
 
-            if (missedForLoad != null) {
-                AffinityTopologyVersion topVer = topologyVersionSnapshot();
+                if (missedForLoad != null) {
+                    AffinityTopologyVersion topVer = topologyVersionSnapshot();
 
-                if (topVer == null)
-                    topVer = entryTopVer;
+                    if (topVer == null)
+                        topVer = entryTopVer;
 
-                IgniteInternalFuture<Void> loadFut = loadMissing(cacheCtx,
-                    topVer != null ? topVer : topologyVersion(),
-                    missedForLoad,
-                    filter,
-                    ret,
-                    needReadVer,
-                    singleRmv,
-                    hasFilters,
-                    /*read through*/(invokeMap != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore,
-                    retval,
-                    keepBinary,
-                    recovery,
-                    expiryPlc);
+                    IgniteInternalFuture<Void> loadFut = loadMissing(cacheCtx,
+                        topVer != null ? topVer : topologyVersion(),
+                        missedForLoad,
+                        filter,
+                        ret,
+                        needReadVer,
+                        singleRmv,
+                        hasFilters,
+                        /*read through*/(invokeMap != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore,
+                        retval,
+                        keepBinary,
+                        recovery,
+                        expiryPlc);
 
-                loadFut.listen(new IgniteInClosure<IgniteInternalFuture<Void>>() {
-                    @Override public void apply(IgniteInternalFuture<Void> fut) {
-                        try {
-                            fut.get();
+                    loadFut.listen(new IgniteInClosure<IgniteInternalFuture<Void>>() {
+                        @Override public void apply(IgniteInternalFuture<Void> fut) {
+                            try {
+                                fut.get();
 
-                            finishFuture(enlistFut, null, true);
-                        }
-                        catch (IgniteCheckedException e) {
-                            finishFuture(enlistFut, e, true);
+                                finishFuture(enlistFut, null, true);
+                            }
+                            catch (IgniteCheckedException e) {
+                                finishFuture(enlistFut, e, true);
+                            }
                         }
-                    }
-                });
+                    });
 
-                return enlistFut;
-            }
+                    return enlistFut;
+                }
 
-            return finishFuture(enlistFut, null, true);
-        }
-        catch (IgniteCheckedException e) {
-            return finishFuture(enlistFut, e, true);
+                return finishFuture(enlistFut, null, true);
+            }
+            catch (IgniteCheckedException e) {
+                return finishFuture(enlistFut, e, true);
+            }
         }
     }
 
@@ -2597,282 +2609,285 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
         assert !F.isEmpty(keys);
         assert keysCnt == keys.size();
 
-        cacheCtx.checkSecurity(SecurityPermission.CACHE_READ);
+        try (TraceSurroundings ignored2 =
+                 MTC.support(context().kernalContext().tracing().create(TX_NEAR_ENLIST_READ, MTC.span()))) {
+            cacheCtx.checkSecurity(SecurityPermission.CACHE_READ);
 
-        boolean single = keysCnt == 1;
+            boolean single = keysCnt == 1;
 
-        Collection<KeyCacheObject> lockKeys = null;
+            Collection<KeyCacheObject> lockKeys = null;
 
-        AffinityTopologyVersion topVer = entryTopVer != null ? entryTopVer : topologyVersion();
+            AffinityTopologyVersion topVer = entryTopVer != null ? entryTopVer : topologyVersion();
 
-        boolean needReadVer = (serializable() && optimistic()) || needVer;
+            boolean needReadVer = (serializable() && optimistic()) || needVer;
 
-        // In this loop we cover only read-committed or optimistic transactions.
-        // Transactions that are pessimistic and not read-committed are covered
-        // outside of this loop.
-        for (KeyCacheObject key : keys) {
-            if (isRollbackOnly())
-                throw timedOut() ? timeoutException() : rollbackException();
+            // In this loop we cover only read-committed or optimistic transactions.
+            // Transactions that are pessimistic and not read-committed are covered
+            // outside of this loop.
+            for (KeyCacheObject key : keys) {
+                if (isRollbackOnly())
+                    throw timedOut() ? timeoutException() : rollbackException();
 
-            if ((pessimistic() || needReadVer) && !readCommitted() && !skipVals)
-                addActiveCache(cacheCtx, recovery);
+                if ((pessimistic() || needReadVer) && !readCommitted() && !skipVals)
+                    addActiveCache(cacheCtx, recovery);
 
-            IgniteTxKey txKey = cacheCtx.txKey(key);
+                IgniteTxKey txKey = cacheCtx.txKey(key);
 
-            // Check write map (always check writes first).
-            IgniteTxEntry txEntry = entry(txKey);
+                // Check write map (always check writes first).
+                IgniteTxEntry txEntry = entry(txKey);
 
-            // Either non-read-committed or there was a previous write.
-            if (txEntry != null) {
-                CacheObject val = txEntry.value();
+                // Either non-read-committed or there was a previous write.
+                if (txEntry != null) {
+                    CacheObject val = txEntry.value();
 
-                if (txEntry.hasValue()) {
-                    if (!F.isEmpty(txEntry.entryProcessors()))
-                        val = txEntry.applyEntryProcessors(val);
+                    if (txEntry.hasValue()) {
+                        if (!F.isEmpty(txEntry.entryProcessors()))
+                            val = txEntry.applyEntryProcessors(val);
 
-                    if (val != null) {
-                        GridCacheVersion ver = null;
+                        if (val != null) {
+                            GridCacheVersion ver = null;
 
-                        if (needVer) {
-                            if (txEntry.op() != READ)
-                                ver = IgniteTxEntry.GET_ENTRY_INVALID_VER_UPDATED;
-                            else {
-                                ver = txEntry.entryReadVersion();
+                            if (needVer) {
+                                if (txEntry.op() != READ)
+                                    ver = IgniteTxEntry.GET_ENTRY_INVALID_VER_UPDATED;
+                                else {
+                                    ver = txEntry.entryReadVersion();
 
-                                if (ver == null && pessimistic()) {
-                                    while (true) {
-                                        try {
-                                            GridCacheEntryEx cached = txEntry.cached();
+                                    if (ver == null && pessimistic()) {
+                                        while (true) {
+                                            try {
+                                                GridCacheEntryEx cached = txEntry.cached();
 
-                                            ver = cached.isNear() ?
-                                                ((GridNearCacheEntry)cached).dhtVersion() : cached.version();
+                                                ver = cached.isNear() ?
+                                                    ((GridNearCacheEntry)cached).dhtVersion() : cached.version();
 
-                                            break;
-                                        }
-                                        catch (GridCacheEntryRemovedException ignored) {
-                                            txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topVer));
+                                                break;
+                                            }
+                                            catch (GridCacheEntryRemovedException ignored) {
+                                                txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topVer));
+                                            }
                                         }
                                     }
-                                }
 
-                                if (ver == null) {
-                                    assert optimistic() && repeatableRead() : this;
+                                    if (ver == null) {
+                                        assert optimistic() && repeatableRead() : this;
 
-                                    ver = IgniteTxEntry.GET_ENTRY_INVALID_VER_AFTER_GET;
+                                        ver = IgniteTxEntry.GET_ENTRY_INVALID_VER_AFTER_GET;
+                                    }
                                 }
+
+                                assert ver != null;
                             }
 
-                            assert ver != null;
+                            cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, false,
+                                ver, 0, 0);
                         }
-
-                        cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, false,
-                            ver, 0, 0);
                     }
-                }
-                else {
-                    assert txEntry.op() == TRANSFORM;
+                    else {
+                        assert txEntry.op() == TRANSFORM;
 
-                    while (true) {
-                        try {
-                            GridCacheVersion readVer = null;
-                            EntryGetResult getRes = null;
+                        while (true) {
+                            try {
+                                GridCacheVersion readVer = null;
+                                EntryGetResult getRes = null;
 
-                            Object transformClo =
-                                (txEntry.op() == TRANSFORM &&
-                                    cctx.gridEvents().isRecordable(EVT_CACHE_OBJECT_READ)) ?
-                                    F.first(txEntry.entryProcessors()) : null;
+                                Object transformClo =
+                                    (txEntry.op() == TRANSFORM &&
+                                        cctx.gridEvents().isRecordable(EVT_CACHE_OBJECT_READ)) ?
+                                        F.first(txEntry.entryProcessors()) : null;
 
-                            if (needVer) {
-                                getRes = txEntry.cached().innerGetVersioned(
-                                    null,
-                                    this,
-                                    /*update-metrics*/true,
-                                    /*event*/!skipVals,
-                                    CU.subjectId(this, cctx),
-                                    transformClo,
-                                    resolveTaskName(),
-                                    null,
-                                    txEntry.keepBinary(),
-                                    null);
+                                if (needVer) {
+                                    getRes = txEntry.cached().innerGetVersioned(
+                                        null,
+                                        this,
+                                        /*update-metrics*/true,
+                                        /*event*/!skipVals,
+                                        CU.subjectId(this, cctx),
+                                        transformClo,
+                                        resolveTaskName(),
+                                        null,
+                                        txEntry.keepBinary(),
+                                        null);
 
-                                if (getRes != null) {
-                                    val = getRes.value();
-                                    readVer = getRes.version();
+                                    if (getRes != null) {
+                                        val = getRes.value();
+                                        readVer = getRes.version();
+                                    }
+                                }
+                                else {
+                                    val = txEntry.cached().innerGet(
+                                        null,
+                                        this,
+                                        /*read-through*/false,
+                                        /*metrics*/true,
+                                        /*event*/!skipVals,
+                                        CU.subjectId(this, cctx),
+                                        transformClo,
+                                        resolveTaskName(),
+                                        null,
+                                        txEntry.keepBinary());
                                 }
-                            }
-                            else {
-                                val = txEntry.cached().innerGet(
-                                    null,
-                                    this,
-                                    /*read-through*/false,
-                                    /*metrics*/true,
-                                    /*event*/!skipVals,
-                                    CU.subjectId(this, cctx),
-                                    transformClo,
-                                    resolveTaskName(),
-                                    null,
-                                    txEntry.keepBinary());
-                            }
-
-                            if (val != null) {
-                                if (!readCommitted() && !skipVals)
-                                    txEntry.readValue(val);
 
-                                if (!F.isEmpty(txEntry.entryProcessors()))
-                                    val = txEntry.applyEntryProcessors(val);
+                                if (val != null) {
+                                    if (!readCommitted() && !skipVals)
+                                        txEntry.readValue(val);
+
+                                    if (!F.isEmpty(txEntry.entryProcessors()))
+                                        val = txEntry.applyEntryProcessors(val);
+
+                                    cacheCtx.addResult(map,
+                                        key,
+                                        val,
+                                        skipVals,
+                                        keepCacheObjects,
+                                        deserializeBinary,
+                                        false,
+                                        getRes,
+                                        readVer,
+                                        0,
+                                        0,
+                                        needVer);
+                                }
+                                else
+                                    missed.put(key, txEntry.cached().version());
 
-                                cacheCtx.addResult(map,
-                                    key,
-                                    val,
-                                    skipVals,
-                                    keepCacheObjects,
-                                    deserializeBinary,
-                                    false,
-                                    getRes,
-                                    readVer,
-                                    0,
-                                    0,
-                                    needVer);
+                                break;
+                            }
+                            catch (GridCacheEntryRemovedException ignored) {
+                                txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topVer));
                             }
-                            else
-                                missed.put(key, txEntry.cached().version());
-
-                            break;
-                        }
-                        catch (GridCacheEntryRemovedException ignored) {
-                            txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topVer));
                         }
                     }
                 }
-            }
-            // First time access within transaction.
-            else {
-                if (lockKeys == null && !skipVals)
-                    lockKeys = single ? Collections.singleton(key) : new ArrayList<KeyCacheObject>(keysCnt);
+                // First time access within transaction.
+                else {
+                    if (lockKeys == null && !skipVals)
+                        lockKeys = single ? Collections.singleton(key) : new ArrayList<KeyCacheObject>(keysCnt);
 
-                if (!single && !skipVals)
-                    lockKeys.add(key);
+                    if (!single && !skipVals)
+                        lockKeys.add(key);
 
-                while (true) {
-                    GridCacheEntryEx entry = entryEx(cacheCtx, txKey, topVer);
+                    while (true) {
+                        GridCacheEntryEx entry = entryEx(cacheCtx, txKey, topVer);
 
-                    try {
-                        GridCacheVersion ver = entry.version();
+                        try {
+                            GridCacheVersion ver = entry.version();
 
-                        CacheObject val = null;
-                        GridCacheVersion readVer = null;
-                        EntryGetResult getRes = null;
+                            CacheObject val = null;
+                            GridCacheVersion readVer = null;
+                            EntryGetResult getRes = null;
 
                         if ((!pessimistic() || (readCommitted() && !skipVals)) && !readRepair) {
                             IgniteCacheExpiryPolicy accessPlc =
                                 optimistic() ? accessPolicy(cacheCtx, txKey, expiryPlc) : null;
 
-                            if (needReadVer) {
-                                getRes = primaryLocal(entry) ?
-                                    entry.innerGetVersioned(
+                                if (needReadVer) {
+                                    getRes = primaryLocal(entry) ?
+                                        entry.innerGetVersioned(
+                                            null,
+                                            this,
+                                            /*metrics*/true,
+                                            /*event*/true,
+                                            CU.subjectId(this, cctx),
+                                            null,
+                                            resolveTaskName(),
+                                            accessPlc,
+                                            !deserializeBinary,
+                                            null) : null;
+
+                                    if (getRes != null) {
+                                        val = getRes.value();
+                                        readVer = getRes.version();
+                                    }
+                                }
+                                else {
+                                    val = entry.innerGet(
                                         null,
                                         this,
+                                        /*read-through*/false,
                                         /*metrics*/true,
-                                        /*event*/true,
+                                        /*event*/!skipVals,
                                         CU.subjectId(this, cctx),
                                         null,
                                         resolveTaskName(),
                                         accessPlc,
-                                        !deserializeBinary,
-                                        null) : null;
-
-                                if (getRes != null) {
-                                    val = getRes.value();
-                                    readVer = getRes.version();
+                                        !deserializeBinary);
                                 }
-                            }
-                            else {
-                                val = entry.innerGet(
-                                    null,
-                                    this,
-                                    /*read-through*/false,
-                                    /*metrics*/true,
-                                    /*event*/!skipVals,
-                                    CU.subjectId(this, cctx),
-                                    null,
-                                    resolveTaskName(),
-                                    accessPlc,
-                                    !deserializeBinary);
-                            }
 
-                            if (val != null) {
-                                cacheCtx.addResult(map,
-                                    key,
-                                    val,
-                                    skipVals,
-                                    keepCacheObjects,
-                                    deserializeBinary,
-                                    false,
-                                    getRes,
-                                    readVer,
-                                    0,
-                                    0,
-                                    needVer);
+                                if (val != null) {
+                                    cacheCtx.addResult(map,
+                                        key,
+                                        val,
+                                        skipVals,
+                                        keepCacheObjects,
+                                        deserializeBinary,
+                                        false,
+                                        getRes,
+                                        readVer,
+                                        0,
+                                        0,
+                                        needVer);
+                                }
+                                else
+                                    missed.put(key, ver);
                             }
                             else
+                                // We must wait for the lock in pessimistic mode.
                                 missed.put(key, ver);
-                        }
-                        else
-                            // We must wait for the lock in pessimistic mode.
-                            missed.put(key, ver);
-
-                        if (!readCommitted() && !skipVals) {
-                            txEntry = addEntry(READ,
-                                val,
-                                null,
-                                null,
-                                entry,
-                                expiryPlc,
-                                null,
-                                true,
-                                -1L,
-                                -1L,
-                                null,
-                                skipStore,
-                                !deserializeBinary,
-                                CU.isNearEnabled(cacheCtx));
 
-                            // As optimization, mark as checked immediately
-                            // for non-pessimistic if value is not null.
-                            if (val != null && !pessimistic()) {
-                                txEntry.markValid();
+                            if (!readCommitted() && !skipVals) {
+                                txEntry = addEntry(READ,
+                                    val,
+                                    null,
+                                    null,
+                                    entry,
+                                    expiryPlc,
+                                    null,
+                                    true,
+                                    -1L,
+                                    -1L,
+                                    null,
+                                    skipStore,
+                                    !deserializeBinary,
+                                    CU.isNearEnabled(cacheCtx));
 
-                                if (needReadVer) {
-                                    assert readVer != null;
+                                // As optimization, mark as checked immediately
+                                // for non-pessimistic if value is not null.
+                                if (val != null && !pessimistic()) {
+                                    txEntry.markValid();
 
-                                    txEntry.entryReadVersion(readVer);
+                                    if (needReadVer) {
+                                        assert readVer != null;
+
+                                        txEntry.entryReadVersion(readVer);
+                                    }
                                 }
                             }
-                        }
 
-                        break; // While.
-                    }
-                    catch (GridCacheEntryRemovedException ignored) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry in transaction getAllAsync(..) (will retry): " + key);
-                    }
-                    finally {
-                        if (entry != null && readCommitted()) {
-                            if (cacheCtx.isNear()) {
-                                if (cacheCtx.affinity().partitionBelongs(cacheCtx.localNode(), entry.partition(), topVer)) {
-                                    if (entry.markObsolete(xidVer))
-                                        cacheCtx.cache().removeEntry(entry);
+                            break; // While.
+                        }
+                        catch (GridCacheEntryRemovedException ignored) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got removed entry in transaction getAllAsync(..) (will retry): " + key);
+                        }
+                        finally {
+                            if (entry != null && readCommitted()) {
+                                if (cacheCtx.isNear()) {
+                                    if (cacheCtx.affinity().partitionBelongs(cacheCtx.localNode(), entry.partition(), topVer)) {
+                                        if (entry.markObsolete(xidVer))
+                                            cacheCtx.cache().removeEntry(entry);
+                                    }
                                 }
+                                else
+                                    entry.touch();
                             }
-                            else
-                                entry.touch();
                         }
                     }
                 }
             }
-        }
 
-        return lockKeys != null ? lockKeys : Collections.<KeyCacheObject>emptyList();
+            return lockKeys != null ? lockKeys : Collections.<KeyCacheObject>emptyList();
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
index 6bbafc7..bccbb11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
@@ -770,7 +770,8 @@ public class MvccUtils {
             cctx == null || !cctx.skipStore(),
             true,
             0,
-            null
+            null,
+            false
         );
 
         tx.syncMode(FULL_SYNC);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 67f9e18..ebea0719 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -92,6 +92,8 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.processors.tracing.NoopTracing;
+import org.apache.ignite.internal.processors.tracing.Tracing;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -337,6 +339,11 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     }
 
     /** {@inheritDoc} */
+    @Override public Tracing tracing() {
+        return new NoopTracing();
+    }
+
+    /** {@inheritDoc} */
     @Override public GridTaskSessionProcessor session() {
         return null;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
index 2a7999f..b6ce2a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.IgniteTransactionsEx;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -36,6 +37,8 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionMetrics;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX;
+
 /**
  * Grid transactions implementation.
  */
@@ -46,13 +49,17 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     /** Label. */
     private String lb;
 
+    /** Tracing enabled flag. */
+    private boolean tracingEnabled;
+
     /**
      * @param cctx Cache shared context.
      * @param lb Label.
      */
-    public IgniteTransactionsImpl(GridCacheSharedContext<K, V> cctx, @Nullable String lb) {
+    public IgniteTransactionsImpl(GridCacheSharedContext<K, V> cctx, @Nullable String lb, boolean tracingEnabled) {
         this.cctx = cctx;
         this.lb = lb;
+        this.tracingEnabled = tracingEnabled;
     }
 
     /** {@inheritDoc} */
@@ -160,6 +167,18 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     ) {
         cctx.kernalContext().gateway().readLock();
 
+        MTC.supportInitial(cctx.kernalContext().tracing().create(
+            TX,
+            null,
+            lb));
+
+        MTC.span().addTag("isolation", isolation::name);
+        MTC.span().addTag("concurrency", concurrency::name);
+        MTC.span().addTag("timeout", () -> String.valueOf(timeout));
+
+        if (lb != null)
+            MTC.span().addTag("label", () -> lb);
+
         try {
             GridNearTxLocal tx = cctx.tm().userTx(sysCacheCtx);
 
@@ -177,11 +196,11 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
                 true,
                 null,
                 txSize,
-                lb
+                lb,
+                tracingEnabled
             );
 
             assert tx != null;
-
             return tx;
         }
         finally {
@@ -223,7 +242,12 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     @Override public IgniteTransactions withLabel(String lb) {
         A.notNull(lb, "label should not be empty.");
 
-        return new IgniteTransactionsImpl<>(cctx, lb);
+        return new IgniteTransactionsImpl<>(cctx, lb, tracingEnabled);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteTransactions withTracing() {
+        return new IgniteTransactionsImpl<>(cctx, lb, true);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index a2bd1e8..ad43c64 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -83,6 +83,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.EnlistOperation;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
@@ -110,6 +111,16 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRA
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.RENTING;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.USER_FINISH;
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_FINISH_REQ;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_FINISH_RESP;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_PREPARE_REQ;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_PREPARE_RESP;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_PROCESS_DHT_FINISH_REQ;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_PROCESS_DHT_FINISH_RESP;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_PROCESS_DHT_ONE_PHASE_COMMIT_ACK_REQ;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_PROCESS_DHT_PREPARE_REQ;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_PROCESS_DHT_PREPARE_RESP;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
@@ -140,24 +151,27 @@ public class IgniteTxHandler {
      * @param req Request.
      */
     private void processNearTxPrepareRequest(UUID nearNodeId, GridNearTxPrepareRequest req) {
-        if (txPrepareMsgLog.isDebugEnabled()) {
-            txPrepareMsgLog.debug("Received near prepare request [txId=" + req.version() +
-                ", node=" + nearNodeId + ']');
-        }
-
-        ClusterNode nearNode = ctx.node(nearNodeId);
-
-        if (nearNode == null) {
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_NEAR_PREPARE_REQ, MTC.span()))) {
             if (txPrepareMsgLog.isDebugEnabled()) {
-                txPrepareMsgLog.debug("Received near prepare from node that left grid (will ignore) [" +
-                    "txId=" + req.version() +
+                txPrepareMsgLog.debug("Received near prepare request [txId=" + req.version() +
                     ", node=" + nearNodeId + ']');
             }
 
-            return;
-        }
+            ClusterNode nearNode = ctx.node(nearNodeId);
 
-        processNearTxPrepareRequest0(nearNode, req);
+            if (nearNode == null) {
+                if (txPrepareMsgLog.isDebugEnabled()) {
+                    txPrepareMsgLog.debug("Received near prepare from node that left grid (will ignore) [" +
+                        "txId=" + req.version() +
+                        ", node=" + nearNodeId + ']');
+                }
+
+                return;
+            }
+
+            processNearTxPrepareRequest0(nearNode, req);
+        }
     }
 
     /**
@@ -750,27 +764,31 @@ public class IgniteTxHandler {
      * @param res Response.
      */
     private void processNearTxPrepareResponse(UUID nodeId, GridNearTxPrepareResponse res) {
-        if (txPrepareMsgLog.isDebugEnabled())
-            txPrepareMsgLog.debug("Received near prepare response [txId=" + res.version() + ", node=" + nodeId + ']');
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_NEAR_PREPARE_RESP, MTC.span()))) {
+            if (txPrepareMsgLog.isDebugEnabled())
+                txPrepareMsgLog.debug("Received near prepare response [txId=" + res.version() + ", node=" +
+                    nodeId + ']');
 
-        GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)ctx.mvcc()
-            .<IgniteInternalTx>versionedFuture(res.version(), res.futureId());
+            GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)ctx.mvcc()
+                .<IgniteInternalTx>versionedFuture(res.version(), res.futureId());
 
-        if (fut == null) {
-            U.warn(log, "Failed to find future for near prepare response [txId=" + res.version() +
-                ", node=" + nodeId +
-                ", res=" + res + ']');
+            if (fut == null) {
+                U.warn(log, "Failed to find future for near prepare response [txId=" + res.version() +
+                    ", node=" + nodeId +
+                    ", res=" + res + ']');
 
-            return;
-        }
+                return;
+            }
 
-        IgniteInternalTx tx = fut.tx();
+            IgniteInternalTx tx = fut.tx();
 
-        assert tx != null;
+            assert tx != null;
 
-        res.txState(tx.txState());
+            res.txState(tx.txState());
 
-        fut.onResult(nodeId, res);
+            fut.onResult(nodeId, res);
+        }
     }
 
     /**
@@ -778,22 +796,25 @@ public class IgniteTxHandler {
      * @param res Response.
      */
     private void processNearTxFinishResponse(UUID nodeId, GridNearTxFinishResponse res) {
-        if (txFinishMsgLog.isDebugEnabled())
-            txFinishMsgLog.debug("Received near finish response [txId=" + res.xid() + ", node=" + nodeId + ']');
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_NEAR_FINISH_RESP, MTC.span()))) {
+            if (txFinishMsgLog.isDebugEnabled())
+                txFinishMsgLog.debug("Received near finish response [txId=" + res.xid() + ", node=" + nodeId + ']');
 
-        GridNearTxFinishFuture fut = (GridNearTxFinishFuture)ctx.mvcc().<IgniteInternalTx>future(res.futureId());
+            GridNearTxFinishFuture fut = (GridNearTxFinishFuture)ctx.mvcc().<IgniteInternalTx>future(res.futureId());
 
-        if (fut == null) {
-            if (txFinishMsgLog.isDebugEnabled()) {
-                txFinishMsgLog.debug("Failed to find future for near finish response [txId=" + res.xid() +
-                    ", node=" + nodeId +
-                    ", res=" + res + ']');
+            if (fut == null) {
+                if (txFinishMsgLog.isDebugEnabled()) {
+                    txFinishMsgLog.debug("Failed to find future for near finish response [txId=" + res.xid() +
+                        ", node=" + nodeId +
+                        ", res=" + res + ']');
+                }
+
+                return;
             }
 
-            return;
+            fut.onResult(nodeId, res);
         }
-
-        fut.onResult(nodeId, res);
     }
 
     /**
@@ -801,28 +822,33 @@ public class IgniteTxHandler {
      * @param res Response.
      */
     private void processDhtTxPrepareResponse(UUID nodeId, GridDhtTxPrepareResponse res) {
-        GridDhtTxPrepareFuture fut = (GridDhtTxPrepareFuture)ctx.mvcc().versionedFuture(res.version(), res.futureId());
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_PROCESS_DHT_PREPARE_RESP, MTC.span()))) {
+            GridDhtTxPrepareFuture fut =
+                (GridDhtTxPrepareFuture)ctx.mvcc().versionedFuture(res.version(), res.futureId());
 
-        if (fut == null) {
-            if (txPrepareMsgLog.isDebugEnabled()) {
-                txPrepareMsgLog.debug("Failed to find future for dht prepare response [txId=null" +
-                    ", dhtTxId=" + res.version() +
-                    ", node=" + nodeId +
-                    ", res=" + res + ']');
-            }
+            if (fut == null) {
+                if (txPrepareMsgLog.isDebugEnabled()) {
+                    txPrepareMsgLog.debug("Failed to find future for dht prepare response [txId=null" +
+                        ", dhtTxId=" + res.version() +
+                        ", node=" + nodeId +
+                        ", res=" + res + ']');
+                }
 
-            return;
-        }
-        else if (txPrepareMsgLog.isDebugEnabled())
-            txPrepareMsgLog.debug("Received dht prepare response [txId=" + fut.tx().nearXidVersion() + ", node=" + nodeId + ']');
+                return;
+            }
+            else if (txPrepareMsgLog.isDebugEnabled())
+                txPrepareMsgLog.debug("Received dht prepare response [txId=" + fut.tx().nearXidVersion() +
+                    ", node=" + nodeId + ']');
 
-        IgniteInternalTx tx = fut.tx();
+            IgniteInternalTx tx = fut.tx();
 
-        assert tx != null;
+            assert tx != null;
 
-        res.txState(tx.txState());
+            res.txState(tx.txState());
 
-        fut.onResult(nodeId, res);
+            fut.onResult(nodeId, res);
+        }
     }
 
     /**
@@ -830,50 +856,53 @@ public class IgniteTxHandler {
      * @param res Response.
      */
     private void processDhtTxFinishResponse(UUID nodeId, GridDhtTxFinishResponse res) {
-        assert nodeId != null;
-        assert res != null;
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_PROCESS_DHT_FINISH_RESP, MTC.span()))) {
+            assert nodeId != null;
+            assert res != null;
 
-        if (res.checkCommitted()) {
-            GridNearTxFinishFuture fut = (GridNearTxFinishFuture)ctx.mvcc().<IgniteInternalTx>future(res.futureId());
+            if (res.checkCommitted()) {
+                GridNearTxFinishFuture fut = (GridNearTxFinishFuture)ctx.mvcc().<IgniteInternalTx>future(res.futureId());
 
-            if (fut == null) {
-                if (txFinishMsgLog.isDebugEnabled()) {
-                    txFinishMsgLog.debug("Failed to find future for dht finish check committed response [txId=null" +
+                if (fut == null) {
+                    if (txFinishMsgLog.isDebugEnabled()) {
+                        txFinishMsgLog.debug("Failed to find future for dht finish check committed response [txId=null" +
+                            ", dhtTxId=" + res.xid() +
+                            ", node=" + nodeId +
+                            ", res=" + res + ']');
+                    }
+
+                    return;
+                }
+                else if (txFinishMsgLog.isDebugEnabled()) {
+                    txFinishMsgLog.debug("Received dht finish check committed response [txId=" + fut.tx().nearXidVersion() +
                         ", dhtTxId=" + res.xid() +
-                        ", node=" + nodeId +
-                        ", res=" + res + ']');
+                        ", node=" + nodeId + ']');
                 }
 
-                return;
-            }
-            else if (txFinishMsgLog.isDebugEnabled()) {
-                txFinishMsgLog.debug("Received dht finish check committed response [txId=" + fut.tx().nearXidVersion() +
-                    ", dhtTxId=" + res.xid() +
-                    ", node=" + nodeId + ']');
+                fut.onResult(nodeId, res);
             }
+            else {
+                GridDhtTxFinishFuture fut = (GridDhtTxFinishFuture)ctx.mvcc().<IgniteInternalTx>future(res.futureId());
 
-            fut.onResult(nodeId, res);
-        }
-        else {
-            GridDhtTxFinishFuture fut = (GridDhtTxFinishFuture)ctx.mvcc().<IgniteInternalTx>future(res.futureId());
+                if (fut == null) {
+                    if (txFinishMsgLog.isDebugEnabled()) {
+                        txFinishMsgLog.debug("Failed to find future for dht finish response [txId=null" +
+                            ", dhtTxId=" + res.xid() +
+                            ", node=" + nodeId +
+                            ", res=" + res);
+                    }
 
-            if (fut == null) {
-                if (txFinishMsgLog.isDebugEnabled()) {
-                    txFinishMsgLog.debug("Failed to find future for dht finish response [txId=null" +
+                    return;
+                }
+                else if (txFinishMsgLog.isDebugEnabled()) {
+                    txFinishMsgLog.debug("Received dht finish response [txId=" + fut.tx().nearXidVersion() +
                         ", dhtTxId=" + res.xid() +
-                        ", node=" + nodeId +
-                        ", res=" + res);
+                        ", node=" + nodeId + ']');
                 }
 
-                return;
+                fut.onResult(nodeId, res);
             }
-            else if (txFinishMsgLog.isDebugEnabled()) {
-                txFinishMsgLog.debug("Received dht finish response [txId=" + fut.tx().nearXidVersion() +
-                    ", dhtTxId=" + res.xid() +
-                    ", node=" + nodeId + ']');
-            }
-
-            fut.onResult(nodeId, res);
         }
     }
 
@@ -886,16 +915,20 @@ public class IgniteTxHandler {
         UUID nodeId,
         GridNearTxFinishRequest req
     ) {
-        if (txFinishMsgLog.isDebugEnabled())
-            txFinishMsgLog.debug("Received near finish request [txId=" + req.version() + ", node=" + nodeId + ']');
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_NEAR_FINISH_REQ, MTC.span()))) {
+            if (txFinishMsgLog.isDebugEnabled())
+                txFinishMsgLog.debug("Received near finish request [txId=" + req.version() + ", node=" + nodeId +
+                    ']');
 
-        IgniteInternalFuture<IgniteInternalTx> fut = finish(nodeId, null, req);
+            IgniteInternalFuture<IgniteInternalTx> fut = finish(nodeId, null, req);
 
-        assert req.txState() != null || fut == null || fut.error() != null ||
-            (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null) :
-            "[req=" + req + ", fut=" + fut + "]";
+            assert req.txState() != null || fut == null || fut.error() != null ||
+                (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null) :
+                "[req=" + req + ", fut=" + fut + "]";
 
-        return fut;
+            return fut;
+        }
     }
 
     /**
@@ -1155,142 +1188,145 @@ public class IgniteTxHandler {
      * @param req Request.
      */
     private void processDhtTxPrepareRequest(final UUID nodeId, final GridDhtTxPrepareRequest req) {
-        if (txPrepareMsgLog.isDebugEnabled()) {
-            txPrepareMsgLog.debug("Received dht prepare request [txId=" + req.nearXidVersion() +
-                ", dhtTxId=" + req.version() +
-                ", node=" + nodeId + ']');
-        }
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_PROCESS_DHT_PREPARE_REQ, MTC.span()))) {
+            if (txPrepareMsgLog.isDebugEnabled()) {
+                txPrepareMsgLog.debug("Received dht prepare request [txId=" + req.nearXidVersion() +
+                    ", dhtTxId=" + req.version() +
+                    ", node=" + nodeId + ']');
+            }
 
-        assert nodeId != null;
-        assert req != null;
+            assert nodeId != null;
+            assert req != null;
 
-        assert req.transactionNodes() != null;
+            assert req.transactionNodes() != null;
 
-        GridDhtTxRemote dhtTx = null;
-        GridNearTxRemote nearTx = null;
+            GridDhtTxRemote dhtTx = null;
+            GridNearTxRemote nearTx = null;
 
-        GridDhtTxPrepareResponse res;
+            GridDhtTxPrepareResponse res;
 
-        try {
-            res = new GridDhtTxPrepareResponse(
-                req.partition(),
-                req.version(),
-                req.futureId(),
-                req.miniId(),
-                req.deployInfo() != null);
+            try {
+                res = new GridDhtTxPrepareResponse(
+                    req.partition(),
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    req.deployInfo() != null);
 
-            // Start near transaction first.
-            nearTx = !F.isEmpty(req.nearWrites()) ? startNearRemoteTx(ctx.deploy().globalLoader(), nodeId, req) : null;
-            dhtTx = startRemoteTx(nodeId, req, res);
+                // Start near transaction first.
+                nearTx = !F.isEmpty(req.nearWrites()) ? startNearRemoteTx(ctx.deploy().globalLoader(), nodeId, req) : null;
+                dhtTx = startRemoteTx(nodeId, req, res);
 
-            // Set evicted keys from near transaction.
-            if (nearTx != null)
-                res.nearEvicted(nearTx.evicted());
+                // Set evicted keys from near transaction.
+                if (nearTx != null)
+                    res.nearEvicted(nearTx.evicted());
 
-            List<IgniteTxKey> writesCacheMissed = req.nearWritesCacheMissed();
+                List<IgniteTxKey> writesCacheMissed = req.nearWritesCacheMissed();
 
-            if (writesCacheMissed != null) {
-                Collection<IgniteTxKey> evicted0 = res.nearEvicted();
+                if (writesCacheMissed != null) {
+                    Collection<IgniteTxKey> evicted0 = res.nearEvicted();
 
-                if (evicted0 != null)
-                    writesCacheMissed.addAll(evicted0);
+                    if (evicted0 != null)
+                        writesCacheMissed.addAll(evicted0);
 
-                res.nearEvicted(writesCacheMissed);
-            }
+                    res.nearEvicted(writesCacheMissed);
+                }
 
-            if (dhtTx != null)
-                req.txState(dhtTx.txState());
-            else if (nearTx != null)
-                req.txState(nearTx.txState());
+                if (dhtTx != null)
+                    req.txState(dhtTx.txState());
+                else if (nearTx != null)
+                    req.txState(nearTx.txState());
 
-            if (dhtTx != null && !F.isEmpty(dhtTx.invalidPartitions()))
-                res.invalidPartitionsByCacheId(dhtTx.invalidPartitions());
+                if (dhtTx != null && !F.isEmpty(dhtTx.invalidPartitions()))
+                    res.invalidPartitionsByCacheId(dhtTx.invalidPartitions());
 
-            if (req.onePhaseCommit()) {
-                assert req.last();
+                if (req.onePhaseCommit()) {
+                    assert req.last();
 
-                if (dhtTx != null) {
-                    dhtTx.onePhaseCommit(true);
-                    dhtTx.needReturnValue(req.needReturnValue());
+                    if (dhtTx != null) {
+                        dhtTx.onePhaseCommit(true);
+                        dhtTx.needReturnValue(req.needReturnValue());
 
-                    finish(dhtTx, req);
-                }
+                        finish(dhtTx, req);
+                    }
 
-                if (nearTx != null) {
-                    nearTx.onePhaseCommit(true);
+                    if (nearTx != null) {
+                        nearTx.onePhaseCommit(true);
 
-                    finish(nearTx, req);
+                        finish(nearTx, req);
+                    }
                 }
             }
-        }
-        catch (IgniteCheckedException e) {
-            if (e instanceof IgniteTxRollbackCheckedException)
-                U.error(log, "Transaction was rolled back before prepare completed: " + req, e);
-            else if (e instanceof IgniteTxOptimisticCheckedException) {
-                if (log.isDebugEnabled())
-                    log.debug("Optimistic failure for remote transaction (will rollback): " + req);
-            }
-            else
-                U.error(log, "Failed to process prepare request: " + req, e);
-
-            if (nearTx != null)
-                try {
-                    nearTx.rollbackRemoteTx();
-                }
-                catch (Throwable e1) {
-                    e.addSuppressed(e1);
+            catch (IgniteCheckedException e) {
+                if (e instanceof IgniteTxRollbackCheckedException)
+                    U.error(log, "Transaction was rolled back before prepare completed: " + req, e);
+                else if (e instanceof IgniteTxOptimisticCheckedException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Optimistic failure for remote transaction (will rollback): " + req);
                 }
+                else
+                    U.error(log, "Failed to process prepare request: " + req, e);
 
-            res = new GridDhtTxPrepareResponse(
-                req.partition(),
-                req.version(),
-                req.futureId(),
-                req.miniId(),
-                e,
-                req.deployInfo() != null);
-        }
+                if (nearTx != null)
+                    try {
+                        nearTx.rollbackRemoteTx();
+                    }
+                    catch (Throwable e1) {
+                        e.addSuppressed(e1);
+                    }
 
-        if (req.onePhaseCommit()) {
-            IgniteInternalFuture completeFut;
+                res = new GridDhtTxPrepareResponse(
+                    req.partition(),
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    e,
+                    req.deployInfo() != null);
+            }
 
-            IgniteInternalFuture<IgniteInternalTx> dhtFin = dhtTx == null ?
-                null : dhtTx.done() ? null : dhtTx.finishFuture();
+            if (req.onePhaseCommit()) {
+                IgniteInternalFuture completeFut;
 
-            final IgniteInternalFuture<IgniteInternalTx> nearFin = nearTx == null ?
-                null : nearTx.done() ? null : nearTx.finishFuture();
+                IgniteInternalFuture<IgniteInternalTx> dhtFin = dhtTx == null ?
+                    null : dhtTx.done() ? null : dhtTx.finishFuture();
 
-            if (dhtFin != null && nearFin != null) {
-                GridCompoundFuture fut = new GridCompoundFuture();
+                final IgniteInternalFuture<IgniteInternalTx> nearFin = nearTx == null ?
+                    null : nearTx.done() ? null : nearTx.finishFuture();
 
-                fut.add(dhtFin);
-                fut.add(nearFin);
+                if (dhtFin != null && nearFin != null) {
+                    GridCompoundFuture fut = new GridCompoundFuture();
 
-                fut.markInitialized();
+                    fut.add(dhtFin);
+                    fut.add(nearFin);
 
-                completeFut = fut;
-            }
-            else
-                completeFut = dhtFin != null ? dhtFin : nearFin;
+                    fut.markInitialized();
 
-            if (completeFut != null) {
-                final GridDhtTxPrepareResponse res0 = res;
-                final GridDhtTxRemote dhtTx0 = dhtTx;
-                final GridNearTxRemote nearTx0 = nearTx;
+                    completeFut = fut;
+                }
+                else
+                    completeFut = dhtFin != null ? dhtFin : nearFin;
 
-                completeFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
-                    @Override public void apply(IgniteInternalFuture<IgniteInternalTx> fut) {
-                        sendReply(nodeId, req, res0, dhtTx0, nearTx0);
-                    }
-                });
+                if (completeFut != null) {
+                    final GridDhtTxPrepareResponse res0 = res;
+                    final GridDhtTxRemote dhtTx0 = dhtTx;
+                    final GridNearTxRemote nearTx0 = nearTx;
+
+                    completeFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
+                        @Override public void apply(IgniteInternalFuture<IgniteInternalTx> fut) {
+                            sendReply(nodeId, req, res0, dhtTx0, nearTx0);
+                        }
+                    });
+                }
+                else
+                    sendReply(nodeId, req, res, dhtTx, nearTx);
             }
             else
                 sendReply(nodeId, req, res, dhtTx, nearTx);
-        }
-        else
-            sendReply(nodeId, req, res, dhtTx, nearTx);
 
-        assert req.txState() != null || res.error() != null || (dhtTx == null && nearTx == null) :
-            req + " tx=" + dhtTx + " nearTx=" + nearTx;
+            assert req.txState() != null || res.error() != null || (dhtTx == null && nearTx == null) :
+                req + " tx=" + dhtTx + " nearTx=" + nearTx;
+        }
     }
 
     /**
@@ -1299,14 +1335,17 @@ public class IgniteTxHandler {
      */
     private void processDhtTxOnePhaseCommitAckRequest(final UUID nodeId,
         final GridDhtTxOnePhaseCommitAckRequest req) {
-        assert nodeId != null;
-        assert req != null;
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_PROCESS_DHT_ONE_PHASE_COMMIT_ACK_REQ, MTC.span()))) {
+            assert nodeId != null;
+            assert req != null;
 
-        if (log.isDebugEnabled())
-            log.debug("Processing dht tx one phase commit ack request [nodeId=" + nodeId + ", req=" + req + ']');
+            if (log.isDebugEnabled())
+                log.debug("Processing dht tx one phase commit ack request [nodeId=" + nodeId + ", req=" + req + ']');
 
-        for (GridCacheVersion ver : req.versions())
-            ctx.tm().removeTxReturn(ver);
+            for (GridCacheVersion ver : req.versions())
+                ctx.tm().removeTxReturn(ver);
+        }
     }
 
     /**
@@ -1315,95 +1354,98 @@ public class IgniteTxHandler {
      */
     @SuppressWarnings({"unchecked"})
     private void processDhtTxFinishRequest(final UUID nodeId, final GridDhtTxFinishRequest req) {
-        assert nodeId != null;
-        assert req != null;
+        try (TraceSurroundings ignored =
+                 MTC.support(ctx.kernalContext().tracing().create(TX_PROCESS_DHT_FINISH_REQ, MTC.span()))) {
+            assert nodeId != null;
+            assert req != null;
 
-        if (req.checkCommitted()) {
-            boolean committed = req.waitRemoteTransactions() || !ctx.tm().addRolledbackTx(null, req.version());
+            if (req.checkCommitted()) {
+                boolean committed = req.waitRemoteTransactions() || !ctx.tm().addRolledbackTx(null, req.version());
 
-            if (!committed || req.syncMode() != FULL_SYNC)
-                sendReply(nodeId, req, committed, null);
-            else {
-                IgniteInternalFuture<?> fut = ctx.tm().remoteTxFinishFuture(req.version());
+                if (!committed || req.syncMode() != FULL_SYNC)
+                    sendReply(nodeId, req, committed, null);
+                else {
+                    IgniteInternalFuture<?> fut = ctx.tm().remoteTxFinishFuture(req.version());
 
-                fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                    @Override public void apply(IgniteInternalFuture<?> fut) {
-                        sendReply(nodeId, req, true, null);
-                    }
-                });
+                    fut.listen(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> fut) {
+                            sendReply(nodeId, req, true, null);
+                        }
+                    });
+                }
+
+                return;
             }
 
-            return;
-        }
+            // Always add version to rollback history to prevent races with rollbacks.
+            if (!req.commit())
+                ctx.tm().addRolledbackTx(null, req.version());
 
-        // Always add version to rollback history to prevent races with rollbacks.
-        if (!req.commit())
-            ctx.tm().addRolledbackTx(null, req.version());
+            GridDhtTxRemote dhtTx = ctx.tm().tx(req.version());
+            GridNearTxRemote nearTx = ctx.tm().nearTx(req.version());
 
-        GridDhtTxRemote dhtTx = ctx.tm().tx(req.version());
-        GridNearTxRemote nearTx = ctx.tm().nearTx(req.version());
+            IgniteInternalTx anyTx = U.<IgniteInternalTx>firstNotNull(dhtTx, nearTx);
 
-        IgniteInternalTx anyTx = U.<IgniteInternalTx>firstNotNull(dhtTx, nearTx);
+            final GridCacheVersion nearTxId = anyTx != null ? anyTx.nearXidVersion() : null;
 
-        final GridCacheVersion nearTxId = anyTx != null ? anyTx.nearXidVersion() : null;
-
-        if (txFinishMsgLog.isDebugEnabled())
-            txFinishMsgLog.debug("Received dht finish request [txId=" + nearTxId + ", dhtTxId=" + req.version() +
-                ", node=" + nodeId + ']');
+            if (txFinishMsgLog.isDebugEnabled())
+                txFinishMsgLog.debug("Received dht finish request [txId=" + nearTxId + ", dhtTxId=" + req.version() +
+                    ", node=" + nodeId + ']');
 
-        if (anyTx == null && req.commit())
-            ctx.tm().addCommittedTx(null, req.version(), null);
+            if (anyTx == null && req.commit())
+                ctx.tm().addCommittedTx(null, req.version(), null);
 
-        if (dhtTx != null)
-            finish(nodeId, dhtTx, req);
-        else {
-            try {
-                applyPartitionsUpdatesCounters(req.updateCounters(), !req.commit(), false);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
+            if (dhtTx != null)
+                finish(nodeId, dhtTx, req);
+            else {
+                try {
+                    applyPartitionsUpdatesCounters(req.updateCounters(), !req.commit(), false);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
             }
-        }
 
-        if (nearTx != null)
-            finish(nodeId, nearTx, req);
+            if (nearTx != null)
+                finish(nodeId, nearTx, req);
 
-        if (req.replyRequired()) {
-            IgniteInternalFuture completeFut;
+            if (req.replyRequired()) {
+                IgniteInternalFuture completeFut;
 
-            IgniteInternalFuture<IgniteInternalTx> dhtFin = dhtTx == null ?
-                null : dhtTx.done() ? null : dhtTx.finishFuture();
+                IgniteInternalFuture<IgniteInternalTx> dhtFin = dhtTx == null ?
+                    null : dhtTx.done() ? null : dhtTx.finishFuture();
 
-            final IgniteInternalFuture<IgniteInternalTx> nearFin = nearTx == null ?
-                null : nearTx.done() ? null : nearTx.finishFuture();
+                final IgniteInternalFuture<IgniteInternalTx> nearFin = nearTx == null ?
+                    null : nearTx.done() ? null : nearTx.finishFuture();
 
-            if (dhtFin != null && nearFin != null) {
-                GridCompoundFuture fut = new GridCompoundFuture();
+                if (dhtFin != null && nearFin != null) {
+                    GridCompoundFuture fut = new GridCompoundFuture();
 
-                fut.add(dhtFin);
-                fut.add(nearFin);
+                    fut.add(dhtFin);
+                    fut.add(nearFin);
 
-                fut.markInitialized();
+                    fut.markInitialized();
 
-                completeFut = fut;
-            }
-            else
-                completeFut = dhtFin != null ? dhtFin : nearFin;
+                    completeFut = fut;
+                }
+                else
+                    completeFut = dhtFin != null ? dhtFin : nearFin;
 
-            if (completeFut != null) {
-                completeFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
-                    @Override public void apply(IgniteInternalFuture<IgniteInternalTx> fut) {
-                        sendReply(nodeId, req, true, nearTxId);
-                    }
-                });
+                if (completeFut != null) {
+                    completeFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
+                        @Override public void apply(IgniteInternalFuture<IgniteInternalTx> fut) {
+                            sendReply(nodeId, req, true, nearTxId);
+                        }
+                    });
+                }
+                else
+                    sendReply(nodeId, req, true, nearTxId);
             }
             else
-                sendReply(nodeId, req, true, nearTxId);
-        }
-        else
-            sendReply(nodeId, req, true, null);
+                sendReply(nodeId, req, true, null);
 
-        assert req.txState() != null || (dhtTx == null && nearTx == null) : req + " tx=" + dhtTx + " nearTx=" + nearTx;
+            assert req.txState() != null || (dhtTx == null && nearTx == null) : req + " tx=" + dhtTx + " nearTx=" + nearTx;
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index df31df9..863a1c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -747,7 +747,8 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         boolean storeEnabled,
         Boolean mvccOp,
         int txSize,
-        @Nullable String lb
+        @Nullable String lb,
+        boolean tracingEnabled
     ) {
         assert sysCacheCtx == null || sysCacheCtx.systemTx();
 
@@ -770,7 +771,8 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             subjId,
             taskNameHash,
             lb,
-            txDumpsThrottling
+            txDumpsThrottling,
+            tracingEnabled
         );
 
         if (tx.system()) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
index 486d9ba..d9d6839 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
@@ -28,6 +28,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -45,6 +46,12 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionState;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_CLOSE;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_RESUME;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_ROLLBACK;
+import static org.apache.ignite.internal.processors.tracing.SpanType.TX_SUSPEND;
 import static org.apache.ignite.transactions.TransactionState.SUSPENDED;
 
 /**
@@ -224,16 +231,19 @@ public class TransactionProxyImpl<K, V> implements TransactionProxy, Externaliza
 
     /** {@inheritDoc} */
     @Override public void suspend() throws IgniteException {
-        enter();
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_SUSPEND, MTC.span()))) {
+            enter();
 
-        try {
-            cctx.suspendTx(tx);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-        finally {
-            leave();
+            try {
+                cctx.suspendTx(tx);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+            finally {
+                leave();
+            }
         }
     }
 
@@ -295,95 +305,128 @@ public class TransactionProxyImpl<K, V> implements TransactionProxy, Externaliza
 
     /** {@inheritDoc} */
     @Override public void commit() {
-        enter();
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_COMMIT, MTC.span()))) {
+            enter();
 
-        try {
-            IgniteInternalFuture<IgniteInternalTx> commitFut = cctx.commitTxAsync(tx);
+            try {
+                IgniteInternalFuture<IgniteInternalTx> commitFut = cctx.commitTxAsync(tx);
 
-            if (async)
-                saveFuture(commitFut);
-            else
-                commitFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+                if (async)
+                    saveFuture(commitFut);
+                else
+                    commitFut.get();
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+            finally {
+                leave();
+            }
         }
         finally {
-            leave();
+            MTC.span().end();
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> commitAsync() throws IgniteException {
-        enter();
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_COMMIT, MTC.span()))) {
+            enter();
 
-        try {
-            return (IgniteFuture<Void>)createFuture(cctx.commitTxAsync(tx));
+            try {
+                return (IgniteFuture<Void>)createFuture(cctx.commitTxAsync(tx));
+            }
+            finally {
+                leave();
+            }
         }
         finally {
-            leave();
+            MTC.span().end();
         }
     }
 
     /** {@inheritDoc} */
     @Override public void close() {
-        enter();
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_CLOSE, MTC.span()))) {
+            enter();
 
-        try {
-            cctx.endTx(tx);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+            try {
+                cctx.endTx(tx);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+            finally {
+                leave();
+            }
         }
         finally {
-            leave();
+            MTC.span().end();
         }
     }
 
     /** {@inheritDoc} */
     @Override public void rollback() {
-        enter();
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_ROLLBACK, MTC.span()))) {
+            enter();
 
-        try {
-            IgniteInternalFuture rollbackFut = cctx.rollbackTxAsync(tx);
+            try {
+                IgniteInternalFuture rollbackFut = cctx.rollbackTxAsync(tx);
 
-            if (async)
-                asyncRes = new IgniteFutureImpl(rollbackFut);
-            else
-                rollbackFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+                if (async)
+                    asyncRes = new IgniteFutureImpl(rollbackFut);
+                else
+                    rollbackFut.get();
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+            finally {
+                leave();
+            }
         }
         finally {
-            leave();
+            MTC.span().end();
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> rollbackAsync() throws IgniteException {
-        enter();
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_ROLLBACK, MTC.span()))) {
+            enter();
 
-        try {
-            return (IgniteFuture<Void>)(new IgniteFutureImpl(cctx.rollbackTxAsync(tx)));
+            try {
+                return (IgniteFuture<Void>)(new IgniteFutureImpl(cctx.rollbackTxAsync(tx)));
+            }
+            finally {
+                leave();
+            }
         }
         finally {
-            leave();
+            MTC.span().end();
         }
     }
 
     /** {@inheritDoc} */
     @Override public void resume() throws IgniteException {
-        enter(true);
+        try (TraceSurroundings ignored =
+                 MTC.support(cctx.kernalContext().tracing().create(TX_RESUME, MTC.span()))) {
+            enter(true);
 
-        try {
-            cctx.resumeTx(tx);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-        finally {
-            leave();
+            try {
+                cctx.resumeTx(tx);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+            finally {
+                leave();
+            }
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index f224d55..40ddc6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -1674,7 +1674,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             err != null);
 
         try {
-            ctx.io().sendToGridTopic(node, TOPIC_CONTINUOUS, msg, SYSTEM_POOL, null);
+            ctx.io().sendToGridTopic(node, TOPIC_CONTINUOUS, msg, SYSTEM_POOL);
         }
         catch (ClusterTopologyCheckedException e) {
             if (log.isDebugEnabled())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java
index ef93627..49ce9a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java
@@ -75,7 +75,8 @@ public class ClientTxStartRequest extends ClientRequest {
                 true,
                 null,
                 0,
-                lb
+                lb,
+                false
             );
         }
         finally {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/DeferredSpan.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/DeferredSpan.java
new file mode 100644
index 0000000..af3da2a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/DeferredSpan.java
@@ -0,0 +1,91 @@
+/*
+ * 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.tracing;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.function.Supplier;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.SpanStatus;
+
+/**
+ * Encapsulates concept of a deferred-initialized span. It's used to overcome OpenCensus span implementation, that starts
+ * span immediately after deserialization.
+ */
+public class DeferredSpan implements Span {
+    /** */
+    private byte[] serializedSpan;
+
+    /**
+     * Constructor.
+     *
+     * @param serializedSpan Serialized span bytes.
+     */
+    @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+    public DeferredSpan(byte[] serializedSpan) {
+        this.serializedSpan = serializedSpan;
+    }
+
+    /**
+     * @return Serialized span.
+     */
+    @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+    public byte[] serializedSpan() {
+        return serializedSpan;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span addTag(String tagName, Supplier<String> tagValSupplier) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span addLog(Supplier<String> logDescSupplier) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span setStatus(SpanStatus spanStatus) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span end() {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEnded() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SpanType type() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Scope> includedScopes() {
+        return Collections.emptySet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isChainable(Scope scope) {
+        return false;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/MTC.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/MTC.java
new file mode 100644
index 0000000..8e4457f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/MTC.java
@@ -0,0 +1,118 @@
+/*
+ * 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.tracing;
+
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings.NOOP_UNCLOSED_SURROUNDINGS;
+
+/**
+ * Mapped tracing context.
+ *
+ * Thread local context which holding the information for tracing.
+ */
+public class MTC {
+    /***/
+    private static final Span NOOP_SPAN = NoopSpan.INSTANCE;
+
+    /** Thread local span holder. */
+    private static ThreadLocal<Span> span = ThreadLocal.withInitial(() -> NOOP_SPAN);
+
+    /**
+     * @return Span which corresponded to current thread or null if it doesn't not set.
+     */
+    @NotNull public static Span span() {
+        return span.get();
+    }
+
+    /**
+     * Attach given span to current thread if it isn't null or attach empty span if it is null. Detach given span, close
+     * it and return previous span when {@link TraceSurroundings#close()} would be called.
+     *
+     * @param startSpan Span which should be added to current thread.
+     * @return {@link TraceSurroundings} for manage span life cycle.
+     */
+    public static TraceSurroundings support(Span startSpan) {
+        Span oldSpan = span();
+
+        if (startSpan != null && startSpan != NOOP_SPAN) {
+            span.set(startSpan);
+
+            return new TraceSurroundings(oldSpan, true);
+        }
+        else
+            return oldSpan == NOOP_SPAN ? NOOP_UNCLOSED_SURROUNDINGS : new TraceSurroundings(oldSpan, false);
+    }
+
+    /**
+     * Support initial span.
+     *
+     * @param startSpan Span which should be added to current thread.
+     */
+    public static void supportInitial(Span startSpan) {
+        span.set(startSpan);
+    }
+
+    /**
+     * Attach given span to current thread if it isn't null or attach empty span if it is null.
+     *
+     * @param startSpan Span which should be added to current thread.
+     * @return {@link TraceSurroundings} for manage span life cycle.
+     */
+    public static TraceSurroundings supportContinual(Span startSpan) {
+        Span oldSpan = span();
+
+        if (startSpan != null && startSpan != NOOP_SPAN)
+            span.set(startSpan);
+
+        return new TraceSurroundings(oldSpan, false);
+    }
+
+    /**
+     * Helper for managing of span life cycle. It help to end current span and also reattach previous one to thread after
+     * {@link TraceSurroundings#close()} would be call.
+     */
+    public static class TraceSurroundings implements AutoCloseable {
+        /** Span which should be attached to thread when {@code #close} would be called. */
+        private final Span oldSpan;
+
+        /** {@code true} if current span should be ended at close moment. */
+        private final boolean endRequired;
+
+        /** Precreated instance of current class for performance target. */
+        static final TraceSurroundings NOOP_UNCLOSED_SURROUNDINGS = new TraceSurroundings(NOOP_SPAN, false);
+
+        /**
+         * @param oldSpan Old span for restoring after close.
+         * @param endRequired {@code true} if current span should be ended at close moment.
+         */
+        private TraceSurroundings(Span oldSpan, boolean endRequired) {
+            this.oldSpan = oldSpan;
+            this.endRequired = endRequired;
+        }
+
+        /**
+         * Close life cycle of current span.
+         */
+        @Override public void close() {
+            if (endRequired)
+                span.get().end();
+
+            span.set(oldSpan);
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/NoopSpan.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/NoopSpan.java
new file mode 100644
index 0000000..b1eaa2a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/NoopSpan.java
@@ -0,0 +1,74 @@
+/*
+ * 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.tracing;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.function.Supplier;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.SpanStatus;
+
+/**
+ * Noop and null-safe implementation of Span.
+ */
+public class NoopSpan implements Span {
+    /** Instance. */
+    public static final Span INSTANCE = new NoopSpan();
+
+    /**
+     * Constructor.
+     */
+    private NoopSpan(){
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span addTag(String tagName, Supplier<String> tagValSupplier) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span addLog(Supplier<String> logDescSupplier) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span setStatus(SpanStatus spanStatus) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span end() {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEnded() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SpanType type() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Scope> includedScopes() {
+        return Collections.emptySet();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/NoopTracing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/NoopTracing.java
new file mode 100644
index 0000000..7756375
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/NoopTracing.java
@@ -0,0 +1,76 @@
+/*
+ * 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.tracing;
+
+import org.apache.ignite.internal.processors.tracing.configuration.NoopTracingConfigurationManager;
+import org.apache.ignite.internal.processors.tracing.messages.TraceableMessagesHandler;
+import org.apache.ignite.logger.NullLogger;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Noop implementation of {@link Tracing}.
+ */
+public class NoopTracing implements Tracing {
+    /** Noop serialized span. */
+    public static final byte[] NOOP_SERIALIZED_SPAN = new byte[0];
+
+    /** Traceable messages handler. */
+    private final TraceableMessagesHandler msgHnd;
+
+    /**
+     * Constructor.
+     */
+    public NoopTracing() {
+        msgHnd = new TraceableMessagesHandler(this, new NullLogger());
+    }
+
+    /** {@inheritDoc} */
+    @Override public TraceableMessagesHandler messages() {
+        return msgHnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span create(@NotNull SpanType spanType, @Nullable Span parentSpan) {
+        return NoopSpan.INSTANCE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span create(@NotNull SpanType spanType, @Nullable byte[] serializedParentSpan) {
+        return NoopSpan.INSTANCE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull Span create(
+        @NotNull SpanType spanType,
+        @Nullable Span parentSpan,
+        @Nullable String label) {
+        return NoopSpan.INSTANCE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] serialize(@NotNull Span span) {
+        return NOOP_SERIALIZED_SPAN;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull TracingConfigurationManager configuration() {
+        return NoopTracingConfigurationManager.INSTANCE;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/Span.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/Span.java
new file mode 100644
index 0000000..3fbeb23
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/Span.java
@@ -0,0 +1,85 @@
+/*
+ * 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.tracing;
+
+import java.util.Set;
+import java.util.function.Supplier;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.SpanStatus;
+
+/**
+ * Logical piece of a trace that represents a single operation.
+ * Each unit work is called a Span in a trace.
+ * Spans include metadata about the work, including the time spent in the step (latency),
+ * status, time events, attributes, links.
+ * You can use tracing to debug errors and latency issues in your applications.
+ */
+public interface Span {
+    /**
+     * Adds tag to span with {@code String} value.
+     *
+     * @param tagName Tag name.
+     * @param tagValSupplier Tag value supplier. Supplier is used instead of strict tag value cause of it's lazy nature.
+     *  So that it's possible not to generate String tag value in case of NoopSpan.
+     */
+    Span addTag(String tagName, Supplier<String> tagValSupplier);
+
+    /**
+     * Logs work to span.
+     *
+     * @param logDescSupplier Log description supplier.
+     *  Supplier is used instead of strict log description cause of it's lazy nature.
+     *  So that it's possible not to generate String log description in case of NoopSpan.
+     */
+    Span addLog(Supplier<String> logDescSupplier);
+
+    /**
+     * Explicitly set status for span.
+     *
+     * @param spanStatus Status.
+     */
+    Span setStatus(SpanStatus spanStatus);
+
+    /**
+     * Ends span. This action sets default status if not set and mark the span as ready to be exported.
+     */
+    Span end();
+
+    /**
+     * @return {@code true} if span has already ended.
+     */
+    boolean isEnded();
+
+    /**
+     * @return Type of given span.
+     */
+    SpanType type();
+
+    /**
+     * @return Set of included scopes.
+     */
+    Set<Scope> includedScopes();
+
+    /**
+     * @param scope Chainable scope candidate.
+     * @return {@code true} if given span is chainable with other spans with specified scope.
+     */
+    default boolean isChainable(Scope scope) {
+        return type().scope() == scope || includedScopes().contains(scope);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanImpl.java
new file mode 100644
index 0000000..bf98b79
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanImpl.java
@@ -0,0 +1,102 @@
+/*
+ * 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.tracing;
+
+import java.util.Set;
+import java.util.function.Supplier;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.SpanStatus;
+import org.apache.ignite.spi.tracing.SpiSpecificSpan;
+
+/**
+ * Implementation of a {@link Span}
+ */
+public class SpanImpl implements Span {
+    /** Spi specific span delegate. */
+    private final SpiSpecificSpan spiSpecificSpan;
+
+    /** Span type. */
+    private final SpanType spanType;
+
+    /** Set of extra included scopes for given span in addition to span's scope that is supported by default. */
+    private final Set<Scope> includedScopes;
+
+    /**
+     * Constructor
+     *
+     * @param spiSpecificSpan Spi specific span.
+     * @param spanType Type of a span.
+     * @param includedScopes Set of included scopes.
+     */
+    public SpanImpl(
+        SpiSpecificSpan spiSpecificSpan,
+        SpanType spanType,
+        Set<Scope> includedScopes) {
+        this.spiSpecificSpan = spiSpecificSpan;
+        this.spanType = spanType;
+        this.includedScopes = includedScopes;
+    }
+
+    @Override public Span addTag(String tagName, Supplier<String> tagValSupplier) {
+        spiSpecificSpan.addTag(tagName, tagValSupplier.get());
+
+        return this;
+    }
+
+    @Override public Span addLog(Supplier<String> logDescSupplier) {
+        spiSpecificSpan.addLog(logDescSupplier.get());
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span setStatus(SpanStatus spanStatus) {
+        spiSpecificSpan.setStatus(spanStatus);
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Span end() {
+        spiSpecificSpan.end();
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEnded() {
+        return spiSpecificSpan.isEnded();
+    }
+
+    /** {@inheritDoc} */
+    @Override public SpanType type() {
+        return spanType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Scope> includedScopes() {
+        return includedScopes;
+    }
+
+    /**
+     * @return Spi specific span delegate.
+     */
+    public SpiSpecificSpan spiSpecificSpan() {
+        return spiSpecificSpan;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanManager.java
new file mode 100644
index 0000000..010506d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanManager.java
@@ -0,0 +1,73 @@
+/*
+ * 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.tracing;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Manager for {@link Span} instances.
+ */
+public interface SpanManager {
+    /**
+     * Creates Span with given name.
+     *
+     * @param spanType Type of span to create.
+     */
+    default Span create(@NotNull SpanType spanType) {
+        return create(spanType, (Span)null);
+    }
+
+    /**
+     * Creates Span given name and explicit parent.
+     *
+     * @param spanType Type of span to create.
+     * @param parentSpan Parent span.
+     * @return Created span.
+     */
+    Span create(@NotNull SpanType spanType, @Nullable Span parentSpan);
+
+    /**
+     * Creates Span given name and explicit parent.
+     *
+     * @param spanType Type of span to create.
+     * @param serializedParentSpan Parent span as serialized bytes.
+     * @return Created span.
+     */
+    Span create(@NotNull SpanType spanType, @Nullable byte[] serializedParentSpan);
+
+    /**
+     * Creates Span given name and explicit parent.
+     *
+     * @param spanType Type of span to create.
+     * @param parentSpan Parent span.
+     * @param lb Label.
+     * @return Created span.
+     */
+    @NotNull Span create(
+        @NotNull SpanType spanType,
+        @Nullable Span parentSpan,
+        @Nullable String lb);
+
+    /**
+     * Serializes span to byte array to send context over network.
+     *
+     * @param span Span.
+     */
+    byte[] serialize(@NotNull Span span);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanTags.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanTags.java
new file mode 100644
index 0000000..d66b324
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanTags.java
@@ -0,0 +1,97 @@
+/*
+ * 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.tracing;
+
+/**
+ * List of tags that can be used to decorate spans.
+ */
+public class SpanTags {
+    /** Tag parts default delimiter. */
+    private static final String TAG_PARTS_DELIMITER = ".";
+
+    /**
+     * List of basic tags. Can be combined together to get more composite tags.
+     * Don't forget to add new tags here and use constant reference instead of raw string creation.
+     * Frequently used composite tags can be also declared here.
+     */
+
+    /** */
+    public static final String NODE = "node";
+
+    /** */
+    public static final String ID = "id";
+
+    /** */
+    public static final String ORDER = "order";
+
+    /** */
+    public static final String EVENT = "event";
+
+    /** */
+    public static final String NAME = "name";
+
+    /** */
+    public static final String TYPE = "type";
+
+    /** */
+    public static final String INITIAL = "initial";
+
+    /** */
+    public static final String RESULT = "result";
+
+    /** */
+    public static final String ERROR = "error";
+
+    /** */
+    public static final String EXCHANGE = "exchange";
+
+    /** */
+    public static final String CONSISTENT_ID = "consistent.id";
+
+    /** */
+    public static final String TOPOLOGY_VERSION = "topology.version";
+
+    /** */
+    public static final String MAJOR = "major";
+
+    /** */
+    public static final String MINOR = "minor";
+
+    /** */
+    public static final String EVENT_NODE = tag(EVENT, NODE);
+
+    /** */
+    public static final String NODE_ID = tag(NODE, ID);
+
+    /** */
+    public static final String MESSAGE = "message";
+
+    /** */
+    public static final String MESSAGE_CLASS = "message.class";
+
+    /** */
+    private SpanTags() {}
+
+    /**
+     * @param tagParts String parts of composite tag.
+     * @return Composite tag with given parts joined using delimiter.
+     */
+    public static String tag(String... tagParts) {
+        return String.join(TAG_PARTS_DELIMITER, tagParts);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanType.java
new file mode 100644
index 0000000..6aa282e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanType.java
@@ -0,0 +1,254 @@
+/*
+ * 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.tracing;
+
+import org.apache.ignite.spi.tracing.Scope;
+
+/**
+ * List of span type names used in appropriate sub-systems.
+ */
+public enum SpanType {
+    // Discovery traces.
+    /** Node join request. */
+    DISCOVERY_NODE_JOIN_REQUEST(Scope.DISCOVERY, "discovery.node.join.request", 1, true),
+
+    /** Node join add. */
+    DISCOVERY_NODE_JOIN_ADD(Scope.DISCOVERY, "discovery.node.join.add", 2),
+
+    /** Node join finish. */
+    DISCOVERY_NODE_JOIN_FINISH(Scope.DISCOVERY, "discovery.node.join.finish", 3),
+
+    /** Node failed. */
+    DISCOVERY_NODE_FAILED(Scope.DISCOVERY, "discovery.node.failed", 4, true),
+
+    /** Node left. */
+    DISCOVERY_NODE_LEFT(Scope.DISCOVERY, "discovery.node.left", 5, true),
+
+    /** Custom event. */
+    DISCOVERY_CUSTOM_EVENT(Scope.DISCOVERY, "discovery.custom.event", 6, true),
+
+    /** Exchange future. */
+    EXCHANGE_FUTURE(Scope.DISCOVERY, "exchange.future", 7),
+
+    /** Affinity calculation. */
+    AFFINITY_CALCULATION(Scope.DISCOVERY, "affinity.calculation", 8),
+
+    // Communication traces.
+    /** Job execution request. */
+    COMMUNICATION_JOB_EXECUTE_REQUEST(Scope.COMMUNICATION, "communication.job.execute.request", 9),
+
+    /** Job execution response. */
+    COMMUNICATION_JOB_EXECUTE_RESPONSE(Scope.COMMUNICATION, "communication.job.execute.response", 10),
+
+    /** Socket write action. */
+    COMMUNICATION_SOCKET_WRITE(Scope.COMMUNICATION, "socket.write", 11, true),
+
+    /** Socket read action. */
+    COMMUNICATION_SOCKET_READ(Scope.COMMUNICATION, "socket.read", 12),
+
+    /** Process regular. */
+    COMMUNICATION_REGULAR_PROCESS(Scope.COMMUNICATION, "process.regular", 13),
+
+    /** Process ordered. */
+    COMMUNICATION_ORDERED_PROCESS(Scope.COMMUNICATION, "process.ordered", 14),
+
+    // Tx traces.
+    /** Transaction start. */
+    TX(Scope.TX, "transaction", 15, true),
+
+    /** Transaction commit. */
+    TX_COMMIT(Scope.TX, "transactions.commit", 16),
+
+    /** Transaction rollback. */
+    TX_ROLLBACK(Scope.TX, "transactions.rollback", 17),
+
+    /** Transaction close. */
+    TX_CLOSE(Scope.TX, "transactions.close", 18),
+
+    /** Transaction suspend. */
+    TX_SUSPEND(Scope.TX, "transactions.suspend", 19),
+
+    /** Transaction resume. */
+    TX_RESUME(Scope.TX, "transactions.resume", 20),
+
+    /** Transaction near prepare. */
+    TX_NEAR_PREPARE(Scope.TX, "transactions.near.prepare", 21),
+
+    /** Transaction near prepare ondone. */
+    TX_NEAR_PREPARE_ON_DONE(Scope.TX, "transactions.near.prepare.ondone", 22),
+
+    /** Transaction near prepare onerror. */
+    TX_NEAR_PREPARE_ON_ERROR(Scope.TX, "transactions.near.prepare.onerror", 23),
+
+    /** Transaction near prepare ontimeout. */
+    TX_NEAR_PREPARE_ON_TIMEOUT(Scope.TX, "transactions.near.prepare.ontimeout", 24),
+
+    /** Transaction dht prepare. */
+    TX_DHT_PREPARE(Scope.TX, "transactions.dht.prepare", 25),
+
+    /** Transaction dht prepare ondone. */
+    TX_DHT_PREPARE_ON_DONE(Scope.TX, "transactions.dht.prepare.ondone", 26),
+
+    /** Transaction near finish. */
+    TX_NEAR_FINISH(Scope.TX, "transactions.near.finish", 27),
+
+    /** Transaction near finish ondone. */
+    TX_NEAR_FINISH_ON_DONE(Scope.TX, "transactions.near.finish.ondone", 28),
+
+    /** Transaction dht finish. */
+    TX_DHT_FINISH(Scope.TX, "transactions.dht.finish", 29),
+
+    /** Transaction dht finish ondone. */
+    TX_DHT_FINISH_ON_DONE(Scope.TX, "transactions.dht.finish.ondone", 30),
+
+    /** Transaction map proceed. */
+    TX_MAP_PROCEED(Scope.TX, "transactions.lock.map.proceed", 31),
+
+    /** Transaction map proceed. */
+    TX_COLOCATED_LOCK_MAP(Scope.TX, "transactions.colocated.lock.map", 32),
+
+    /** Transaction lock map. */
+    TX_DHT_LOCK_MAP(Scope.TX, "transactions.dht.lock.map", 33),
+
+    /** Transaction near enlist read. */
+    TX_NEAR_ENLIST_READ(Scope.TX, "transactions.near.enlist.read", 34),
+
+    /** Transaction near enlist write. */
+    TX_NEAR_ENLIST_WRITE(Scope.TX, "transactions.near.enlist.write", 35),
+
+    /** Transaction dht process prepare request. */
+    TX_PROCESS_DHT_PREPARE_REQ(Scope.TX, "tx.dht.process.prepare.req", 36),
+
+    /** Transaction dht process finish request. */
+    TX_PROCESS_DHT_FINISH_REQ(Scope.TX, "tx.dht.process.finish.req", 37),
+
+    /** Transaction dht finish response. */
+    TX_PROCESS_DHT_FINISH_RESP(Scope.TX, "tx.dht.process.finish.resp", 38),
+
+    /** Transaction dht one phase commit ack request. */
+    TX_PROCESS_DHT_ONE_PHASE_COMMIT_ACK_REQ(Scope.TX, "tx.dht.process.one-phase-commit-ack.req", 39),
+
+    /** Transaction dht prepare response. */
+    TX_PROCESS_DHT_PREPARE_RESP(Scope.TX, "tx.dht.process.prepare.response", 40),
+
+    /** Transaction near finish request. */
+    TX_NEAR_FINISH_REQ(Scope.TX, "tx.near.process.finish.request", 41),
+
+    /** Transaction near finish  response. */
+    TX_NEAR_FINISH_RESP(Scope.TX, "tx.near.process.finish.response", 42),
+
+    /** Transaction near prepare request. */
+    TX_NEAR_PREPARE_REQ(Scope.TX, "tx.near.process.prepare.request", 43),
+
+    /** Transaction near prepare  response. */
+    TX_NEAR_PREPARE_RESP(Scope.TX, "tx.near.process.prepare.response", 44),
+
+    /** Custom job call. */
+    CUSTOM_JOB_CALL(Scope.COMMUNICATION, "job.call", 45, true);
+
+    /** Scope */
+    private Scope scope;
+
+    /** Trace name. */
+    private String spanName;
+
+    /** Index. */
+    private int idx;
+
+    /** Values. */
+    private static final SpanType[] VALS;
+
+    /** {@code true} if given span is a root span within it's scope. */
+    private boolean rootSpan;
+
+    /**
+     * Constructor.
+     *
+     * @param scope Scope.
+     * @param spanName Span name.
+     * @param idx Index.
+     */
+    SpanType(Scope scope, String spanName, int idx) {
+        this.scope = scope;
+        this.spanName = spanName;
+        this.idx = idx;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param scope Scope.
+     * @param spanName Span name.
+     * @param idx Index.
+     * @param rootSpan Boolean flag, that indicates whether given span is root within it's scope or not.
+     */
+    SpanType(Scope scope, String spanName, int idx, boolean rootSpan) {
+        this(scope, spanName, idx);
+        this.rootSpan = rootSpan;
+    }
+
+    /**
+     * @return Scope.
+     */
+    public Scope scope() {
+        return scope;
+    }
+
+    /**
+     * @return Trace name.
+     */
+    public String spanName() {
+        return spanName;
+    }
+
+    /**
+     * @return idx.
+     */
+    public int index() {
+        return idx;
+    }
+
+    /**
+     * @return Root span.
+     */
+    public boolean rootSpan() {
+        return rootSpan;
+    }
+
+    static {
+        SpanType[] spanTypes = SpanType.values();
+
+        int maxIdx = 0;
+
+        for (SpanType spanType : spanTypes)
+            maxIdx = Math.max(maxIdx, spanType.idx);
+
+        VALS = new SpanType[maxIdx + 1];
+
+        for (SpanType spanType : spanTypes)
+            VALS[spanType.idx] = spanType;
+    }
+
+    /**
+     * @param idx Index.
+     * @return Enum instance based on specified index.
+     */
+    public static SpanType fromIndex(int idx) {
+        return idx < 0 || idx >= VALS.length ? null : VALS[idx];
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/Tracing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/Tracing.java
new file mode 100644
index 0000000..8bc9a80
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/Tracing.java
@@ -0,0 +1,45 @@
+/*
+ * 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.tracing;
+
+import org.apache.ignite.internal.processors.tracing.messages.TraceableMessagesHandler;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Tracing sub-system interface.
+ */
+public interface Tracing extends SpanManager {
+    /**
+     * @return Helper to handle traceable messages.
+     */
+    public TraceableMessagesHandler messages();
+
+    /**
+     * Returns the {@link TracingConfigurationManager} instance that allows to
+     * <ul>
+     *     <li>Configure tracing parameters such as sampling rate for the specific tracing coordinates
+     *          such as scope and label.</li>
+     *     <li>Retrieve the most specific tracing parameters for the specified tracing coordinates (scope and label)</li>
+     *     <li>Restore the tracing parameters for the specified tracing coordinates to the default.</li>
+     *     <li>List all pairs of tracing configuration coordinates and tracing configuration parameters.</li>
+     * </ul>
+     * @return {@link TracingConfigurationManager} instance.
+     */
+    public @NotNull TracingConfigurationManager configuration();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/DistributedTracingConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/DistributedTracingConfiguration.java
new file mode 100644
index 0000000..8624778
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/DistributedTracingConfiguration.java
@@ -0,0 +1,49 @@
+/*
+ * 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.tracing.configuration;
+
+import java.util.HashMap;
+import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
+import org.apache.ignite.internal.processors.configuration.distributed.SimpleDistributedProperty;
+import org.apache.ignite.spi.tracing.TracingConfigurationCoordinates;
+import org.apache.ignite.spi.tracing.TracingConfigurationParameters;
+
+/**
+ * The wrapper of {@code HashMap<TracingConfigurationCoordinates, TracingConfigurationParameters>}
+ * for the distributed metastorage binding.
+ */
+public class DistributedTracingConfiguration
+    extends SimpleDistributedProperty<HashMap<TracingConfigurationCoordinates, TracingConfigurationParameters>> {
+    /** */
+    private static final String TRACING_CONFIGURATION_DISTRIBUTED_METASTORE_KEY = "tr.config";
+
+    /**
+     * Constructor.
+     */
+    public DistributedTracingConfiguration() {
+        super(TRACING_CONFIGURATION_DISTRIBUTED_METASTORE_KEY);
+    }
+
+    /**
+     * @return A new property that is detached from {@link DistributedConfigurationProcessor}.
+     * This means distributed updates are not accessible.
+     */
+    public static DistributedTracingConfiguration detachedProperty() {
+        return new DistributedTracingConfiguration();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/GridTracingConfigurationManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/GridTracingConfigurationManager.java
new file mode 100644
index 0000000..cd03738
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/GridTracingConfigurationManager.java
@@ -0,0 +1,214 @@
+/*
+ * 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.tracing.configuration;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.TracingConfigurationCoordinates;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.apache.ignite.spi.tracing.TracingConfigurationParameters;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Tracing configuration manager implementation that uses distributed meta storage
+ * in order to store tracing configuration.
+ */
+public class GridTracingConfigurationManager implements TracingConfigurationManager {
+    /** Map with default configurations. */
+    private static final Map<TracingConfigurationCoordinates, TracingConfigurationParameters> DEFAULT_CONFIGURATION_MAP;
+
+    /** */
+    public static final String WARNING_MSG_TRACING_CONFIG_UPDATE_FAILED_COORDINATES =
+        "Failed to update tracing configuration for coordinates=[%s].";
+
+    /** */
+    public static final String WARNING_MSG_TRACING_CONFIG_UPDATE_FAILED_SCOPE =
+        "Failed to update tracing configuration for scope=[%s].";
+
+    /** Tracing configuration distributed property. */
+    private final DistributedTracingConfiguration distributedTracingConfiguration =
+        DistributedTracingConfiguration.detachedProperty();
+
+    /** Tracing configuration. */
+    private volatile Map<TracingConfigurationCoordinates, TracingConfigurationParameters> tracingConfiguration =
+        DEFAULT_CONFIGURATION_MAP;
+
+    /** Mutex for updating local tracing configuration. */
+    @GridToStringExclude
+    private final Object mux = new Object();
+
+    static {
+        Map<TracingConfigurationCoordinates, TracingConfigurationParameters> tmpDfltConfigurationMap = new HashMap<>();
+
+        tmpDfltConfigurationMap.put(
+            new TracingConfigurationCoordinates.Builder(Scope.TX).build(),
+            TracingConfigurationManager.DEFAULT_TX_CONFIGURATION);
+
+        tmpDfltConfigurationMap.put(
+            new TracingConfigurationCoordinates.Builder(Scope.COMMUNICATION).build(),
+            TracingConfigurationManager.DEFAULT_COMMUNICATION_CONFIGURATION);
+
+        tmpDfltConfigurationMap.put(
+            new TracingConfigurationCoordinates.Builder(Scope.EXCHANGE).build(),
+            TracingConfigurationManager.DEFAULT_EXCHANGE_CONFIGURATION);
+
+        tmpDfltConfigurationMap.put(
+            new TracingConfigurationCoordinates.Builder(Scope.DISCOVERY).build(),
+            TracingConfigurationManager.DEFAULT_DISCOVERY_CONFIGURATION);
+
+        DEFAULT_CONFIGURATION_MAP = Collections.unmodifiableMap(tmpDfltConfigurationMap);
+    }
+
+    /** Kernal context. */
+    @GridToStringExclude
+    protected final GridKernalContext ctx;
+
+    /** Grid logger. */
+    @GridToStringExclude
+    protected final IgniteLogger log;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    public GridTracingConfigurationManager(@NotNull GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        ctx.internalSubscriptionProcessor().registerDistributedConfigurationListener(dispatcher -> {
+            distributedTracingConfiguration.addListener((name, oldVal, newVal) -> {
+                synchronized (mux) {
+                    if (log.isDebugEnabled())
+                        log.debug("Tracing configuration was updated [oldVal= " + oldVal + ", newVal=" + newVal + "]");
+
+                    if (newVal != null && !newVal.isEmpty())
+                        tracingConfiguration = newVal;
+                }
+            });
+
+            dispatcher.registerProperty(distributedTracingConfiguration);
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void set(
+        @NotNull TracingConfigurationCoordinates coordinates,
+        @NotNull TracingConfigurationParameters parameters)
+    {
+        HashMap<TracingConfigurationCoordinates, TracingConfigurationParameters> newTracingConfiguration =
+            new HashMap<>(tracingConfiguration);
+
+        newTracingConfiguration.put(coordinates, parameters);
+
+        try {
+            distributedTracingConfiguration.propagate(newTracingConfiguration);
+        }
+        catch (IgniteCheckedException e) {
+            String warningMsg = String.format(WARNING_MSG_TRACING_CONFIG_UPDATE_FAILED_COORDINATES, coordinates);
+
+            log.warning(warningMsg, e);
+
+            throw new IgniteException(warningMsg, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull TracingConfigurationParameters get(@NotNull TracingConfigurationCoordinates coordinates) {
+        TracingConfigurationParameters coordinateSpecificParameters = tracingConfiguration.get(coordinates);
+
+        // If parameters for the specified coordinates (both scope and label) were not found use only scope specific one.
+        // If there are no custom scope specific parameters, default one will be used.
+        return coordinateSpecificParameters == null ?
+            tracingConfiguration.get(new TracingConfigurationCoordinates.Builder(coordinates.scope()).build()) :
+            coordinateSpecificParameters;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+    @Override public @NotNull Map<TracingConfigurationCoordinates, TracingConfigurationParameters> getAll(
+        @Nullable Scope scope) {
+        return scope != null ?
+            tracingConfiguration.entrySet().stream().
+                filter(e -> e.getKey().scope() == scope).
+                collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)) :
+            tracingConfiguration;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset(@NotNull TracingConfigurationCoordinates coordinates) {
+        HashMap<TracingConfigurationCoordinates, TracingConfigurationParameters> newTracingConfiguration =
+            new HashMap<>(tracingConfiguration);
+
+        if (coordinates.label() != null)
+            newTracingConfiguration.remove(coordinates);
+        else
+            newTracingConfiguration.put(coordinates, DEFAULT_CONFIGURATION_MAP.get(new TracingConfigurationCoordinates.Builder(coordinates.scope()).build()));
+
+        try {
+            distributedTracingConfiguration.propagate(newTracingConfiguration);
+        }
+        catch (IgniteCheckedException e) {
+            String warningMsg = String.format(WARNING_MSG_TRACING_CONFIG_UPDATE_FAILED_COORDINATES, coordinates);
+
+            log.warning(warningMsg, e);
+
+            throw new IgniteException(warningMsg, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resetAll(@Nullable Scope scope) throws IgniteException {
+        HashMap<TracingConfigurationCoordinates, TracingConfigurationParameters> newTracingConfiguration;
+
+        if (scope != null) {
+            newTracingConfiguration = new HashMap<>(tracingConfiguration.entrySet().stream().
+                filter(e -> e.getKey().scope() != scope).
+                collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+
+            TracingConfigurationCoordinates scopeSpecificCoordinates =
+                new TracingConfigurationCoordinates.Builder(scope).build();
+
+            newTracingConfiguration.put(scopeSpecificCoordinates,
+                DEFAULT_CONFIGURATION_MAP.get(scopeSpecificCoordinates));
+        }
+        else
+            newTracingConfiguration = new HashMap<>(DEFAULT_CONFIGURATION_MAP);
+
+        try {
+            distributedTracingConfiguration.propagate(newTracingConfiguration);
+        }
+        catch (IgniteCheckedException e) {
+            String warningMsg = String.format(WARNING_MSG_TRACING_CONFIG_UPDATE_FAILED_SCOPE, scope);
+
+            log.warning(warningMsg, e);
+
+            throw new IgniteException(warningMsg, e);
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/NoopTracingConfigurationManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/NoopTracingConfigurationManager.java
new file mode 100644
index 0000000..3345148
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/configuration/NoopTracingConfigurationManager.java
@@ -0,0 +1,67 @@
+/*
+ * 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.tracing.configuration;
+
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.spi.tracing.Scope;
+import org.apache.ignite.spi.tracing.TracingConfigurationCoordinates;
+import org.apache.ignite.spi.tracing.TracingConfigurationManager;
+import org.apache.ignite.spi.tracing.TracingConfigurationParameters;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Noop tracing configuration manager.
+ * To be used mainly with {@link org.apache.ignite.internal.processors.tracing.NoopTracing}.
+ */
+public final class NoopTracingConfigurationManager implements TracingConfigurationManager {
+    /** */
+    public static final NoopTracingConfigurationManager INSTANCE = new NoopTracingConfigurationManager();
+
+    /** {@inheritDoc} */
+    @Override public void set(@NotNull TracingConfigurationCoordinates coordinates,
+        @NotNull TracingConfigurationParameters parameters) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull TracingConfigurationParameters get(
+        @NotNull TracingConfigurationCoordinates coordinates) {
+        return TracingConfigurationManager.NOOP_CONFIGURATION;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull Map<TracingConfigurationCoordinates, TracingConfigurationParameters> getAll(
+        @Nullable Scope scope
+    ) {
+        return Collections.emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset(@NotNull TracingConfigurationCoordinates coordinates) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resetAll(@Nullable Scope scope) throws IgniteException {
+        // No-op.
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/SpanContainer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/SpanContainer.java
new file mode 100644
index 0000000..c1f73f4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/SpanContainer.java
@@ -0,0 +1,82 @@
+/*
+ * 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.tracing.messages;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Container to store serialized span context and span that is created from this context.
+ */
+public class SpanContainer implements Serializable {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Serialized span bytes. */
+    private byte[] serializedSpanBytes;
+
+    /** Span. */
+    private transient Span span = NoopSpan.INSTANCE;
+
+    /**
+     * @return Serialized span.
+     */
+    public byte[] serializedSpanBytes() {
+        return serializedSpanBytes;
+    }
+
+    /**
+     * @param serializedSpan Serialized span.
+     */
+    public void serializedSpanBytes(byte[] serializedSpan) {
+        this.serializedSpanBytes = serializedSpan.clone();
+    }
+
+    /**
+     * @return Span span.
+     */
+    public @NotNull Span span() {
+        return span;
+    }
+
+    /**
+     * @param span Span.
+     */
+    public void span(@NotNull Span span) {
+        this.span = span;
+    }
+
+    /**
+     * Restores span field to default value after deserialization.
+     */
+    public Object readResolve() {
+        span = NoopSpan.INSTANCE;
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "SpanContainer{" +
+            "serializedSpanBytes=" + serializedSpanBytes +
+            ", span=" + span +
+            '}';
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/SpanTransport.java
similarity index 50%
copy from modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/SpanTransport.java
index 7188f9f..6466be6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/SpanTransport.java
@@ -15,32 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.discovery.tcp.messages;
-
-import java.util.UUID;
-import org.apache.ignite.internal.util.typedef.internal.S;
+package org.apache.ignite.internal.processors.tracing.messages;
 
 /**
- * Sent by node that is stopping to coordinator across the ring,
- * then sent by coordinator across the ring.
+ * This interface indicates class which able to transfer span.
  */
-@TcpDiscoveryEnsureDelivery
-@TcpDiscoveryRedirectToClient
-public class TcpDiscoveryNodeLeftMessage extends TcpDiscoveryAbstractMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
+public interface SpanTransport {
     /**
-     * Constructor.
+     * Stored span for transferring.
      *
-     * @param creatorNodeId ID of the node that is about to leave the topology.
+     * @param span Binary view of span.
      */
-    public TcpDiscoveryNodeLeftMessage(UUID creatorNodeId) {
-        super(creatorNodeId);
-    }
+    void span(byte[] span);
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(TcpDiscoveryNodeLeftMessage.class, this, "super", super.toString());
-    }
+    /**
+     * @return Binary view of span.
+     */
+    byte[] span();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessage.java
similarity index 50%
copy from modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessage.java
index 7188f9f..389a608 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessage.java
@@ -15,32 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.discovery.tcp.messages;
-
-import java.util.UUID;
-import org.apache.ignite.internal.util.typedef.internal.S;
+package org.apache.ignite.internal.processors.tracing.messages;
 
 /**
- * Sent by node that is stopping to coordinator across the ring,
- * then sent by coordinator across the ring.
+ * Interface for messages that contain trace context for an operation.
  */
-@TcpDiscoveryEnsureDelivery
-@TcpDiscoveryRedirectToClient
-public class TcpDiscoveryNodeLeftMessage extends TcpDiscoveryAbstractMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
+public interface TraceableMessage {
     /**
-     * Constructor.
-     *
-     * @param creatorNodeId ID of the node that is about to leave the topology.
+     * @return Span Container.
      */
-    public TcpDiscoveryNodeLeftMessage(UUID creatorNodeId) {
-        super(creatorNodeId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(TcpDiscoveryNodeLeftMessage.class, this, "super", super.toString());
-    }
+    public SpanContainer spanContainer();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessagesHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessagesHandler.java
new file mode 100644
index 0000000..5189aed
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessagesHandler.java
@@ -0,0 +1,106 @@
+/*
+ * 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.tracing.messages;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.SpanManager;
+
+/**
+ * Helper to handle traceable messages.
+ */
+public class TraceableMessagesHandler {
+    /** Span manager. */
+    private final SpanManager spanMgr;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /**
+     * @param spanMgr Span manager.
+     * @param log Logger.
+     */
+    public TraceableMessagesHandler(SpanManager spanMgr, IgniteLogger log) {
+        this.spanMgr = spanMgr;
+        this.log = log;
+    }
+
+    /**
+     * Called when message is received.
+     * A span with name associated with given message will be created.
+     * from contained serialized span {@link SpanContainer#serializedSpanBytes()}
+     *
+     * @param msg Traceable message.
+     */
+    public void afterReceive(TraceableMessage msg) {
+        if (log.isDebugEnabled())
+            log.debug("Received traceable message: " + msg);
+
+        if (msg.spanContainer().span() == NoopSpan.INSTANCE && msg.spanContainer().serializedSpanBytes() != null)
+            msg.spanContainer().span(
+                spanMgr.create(TraceableMessagesTable.traceName(msg.getClass()), msg.spanContainer().serializedSpanBytes())
+                    .addLog(() -> "Received")
+            );
+    }
+
+    /**
+     * Called when message is going to be send.
+     * A serialized span will be created and attached to {@link TraceableMessage#spanContainer()}.
+     *
+     * @param msg Traceable message.
+     */
+    public void beforeSend(TraceableMessage msg) {
+        if (msg.spanContainer().span() != NoopSpan.INSTANCE && msg.spanContainer().serializedSpanBytes() == null)
+            msg.spanContainer().serializedSpanBytes(spanMgr.serialize(msg.spanContainer().span()));
+    }
+
+    /**
+     * Injects a sub-span to {@code msg} as child span contained in given {@code parent}.
+     *
+     * @param msg Branched message.
+     * @param parent Parent message.
+     * @param <T> Traceable message type.
+     * @return Branched message with span context from parent message.
+     */
+    public <T extends TraceableMessage> T branch(T msg, TraceableMessage parent) {
+        assert parent.spanContainer().span() != null : parent;
+
+        msg.spanContainer().serializedSpanBytes(
+            spanMgr.serialize(parent.spanContainer().span())
+        );
+
+        msg.spanContainer().span(
+            spanMgr.create(TraceableMessagesTable.traceName(msg.getClass()), parent.spanContainer().span())
+                .addLog(() -> "Created")
+        );
+
+        return msg;
+    }
+
+    /**
+     * @param msg Message.
+     */
+    public void finishProcessing(TraceableMessage msg) {
+        if (log.isDebugEnabled())
+            log.debug("Processed traceable message: " + msg);
+
+        if (!msg.spanContainer().span().isEnded())
+            msg.spanContainer().span()
+                .addLog(() -> "Processed")
+                .end();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessagesTable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessagesTable.java
new file mode 100644
index 0000000..e46bd99
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/messages/TraceableMessagesTable.java
@@ -0,0 +1,88 @@
+/*
+ * 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.tracing.messages;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridJobExecuteRequest;
+import org.apache.ignite.internal.GridJobExecuteResponse;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.tracing.SpanType;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryServerOnlyCustomEventMessage;
+
+/**
+ * Mapping from traceable message to appropriate trace.
+ *
+ * @see TraceableMessage inheritors.
+ */
+public class TraceableMessagesTable {
+    /** Message trace lookup table. */
+    private static final Map<Class<?>, SpanType> msgTraceLookupTable = new ConcurrentHashMap<>();
+
+    static {
+        msgTraceLookupTable.put(TcpDiscoveryJoinRequestMessage.class, SpanType.DISCOVERY_NODE_JOIN_REQUEST);
+        msgTraceLookupTable.put(TcpDiscoveryNodeAddedMessage.class, SpanType.DISCOVERY_NODE_JOIN_ADD);
+        msgTraceLookupTable.put(TcpDiscoveryNodeAddFinishedMessage.class, SpanType.DISCOVERY_NODE_JOIN_FINISH);
+        msgTraceLookupTable.put(TcpDiscoveryNodeFailedMessage.class, SpanType.DISCOVERY_NODE_FAILED);
+        msgTraceLookupTable.put(TcpDiscoveryNodeLeftMessage.class, SpanType.DISCOVERY_NODE_LEFT);
+        msgTraceLookupTable.put(TcpDiscoveryCustomEventMessage.class, SpanType.DISCOVERY_CUSTOM_EVENT);
+        msgTraceLookupTable.put(TcpDiscoveryServerOnlyCustomEventMessage.class, SpanType.DISCOVERY_CUSTOM_EVENT);
+        msgTraceLookupTable.put(GridJobExecuteRequest.class, SpanType.COMMUNICATION_JOB_EXECUTE_REQUEST);
+        msgTraceLookupTable.put(GridJobExecuteResponse.class, SpanType.COMMUNICATION_JOB_EXECUTE_RESPONSE);
+    }
+
+    /** */
+    private TraceableMessagesTable() {
+    }
+
+    /**
+     * @param msgCls Traceable message class.
+     * @return Trace name associated with message with given class.
+     */
+    public static SpanType traceName(Class<? extends TraceableMessage> msgCls) {
+        SpanType spanType = msgTraceLookupTable.get(msgCls);
+
+        if (spanType == null)
+            throw new IgniteException("Trace name is not defined for " + msgCls);
+
+        return spanType;
+    }
+
+    /**
+     * @param obj Traceable message object.
+     * @return Trace name associated with message with given class.
+     */
+    public static String traceName(Object obj) {
+        if (obj == null)
+            return "unknown";
+
+        if (obj instanceof GridIoMessage)
+            return traceName(((GridIoMessage)obj).message());
+
+        SpanType val = msgTraceLookupTable.get(obj.getClass());
+
+        return val != null ? val.spanName() : obj.getClass().getSimpleName();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/tracing/TracingSpiType.java
similarity index 52%
copy from modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java
copy to modules/core/src/main/java/org/apache/ignite/internal/tracing/TracingSpiType.java
index 7188f9f..b070e1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/tracing/TracingSpiType.java
@@ -15,32 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.discovery.tcp.messages;
-
-import java.util.UUID;
-import org.apache.ignite.internal.util.typedef.internal.S;
+package org.apache.ignite.internal.tracing;
 
 /**
- * Sent by node that is stopping to coordinator across the ring,
- * then sent by coordinator across the ring.
+ * Type of the tracing spi.
  */
-@TcpDiscoveryEnsureDelivery
-@TcpDiscoveryRedirectToClient
-public class TcpDiscoveryNodeLeftMessage extends TcpDiscoveryAbstractMessage {
+public enum TracingSpiType {
+    /** */
+    NOOP_TRACING_SPI((byte)0),
+
     /** */
-    private static final long serialVersionUID = 0L;
+    OPEN_CENSUS_TRACING_SPI((byte)1);
+
+    /** Byte index of a tracing spi instance. */
+    private final byte idx;
 
     /**
-     * Constructor.
+     * Constrictor
      *
-     * @param creatorNodeId ID of the node that is about to leave the topology.
+     * @param idx Index.
      */
-    public TcpDiscoveryNodeLeftMessage(UUID creatorNodeId) {
-        super(creatorNodeId);
+    TracingSpiType(byte idx) {
+        this.idx = idx;
     }
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(TcpDiscoveryNodeLeftMessage.class, this, "super", super.toString());
+    /**
+     * @return Index.
+     */
+    public byte index() {
+        return idx;
     }
 }
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 222e0a4..a23daf3 100755
--- 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
@@ -1544,6 +1544,18 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @return Stacktrace of current thread as {@link String}.
+     */
+    public static String stackTrace() {
+        GridStringBuilder sb = new GridStringBuilder();
+        long threadId = Thread.currentThread().getId();
+
+        printStackTrace(threadId, sb);
+
+        return sb.toString();
+    }
+
+    /**
      * @return {@code true} if there is java level deadlock.
      */
     public static boolean deadlockPresent() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 21e51b5..027e3cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -57,6 +57,14 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.processors.metric.impl.LongAdderMetric;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.NoopTracing;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.SpanTags;
+import org.apache.ignite.internal.processors.tracing.SpanType;
+import org.apache.ignite.internal.processors.tracing.Tracing;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -82,6 +90,8 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
 import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION;
+import static org.apache.ignite.internal.processors.tracing.SpanType.COMMUNICATION_SOCKET_WRITE;
+import static org.apache.ignite.internal.processors.tracing.messages.TraceableMessagesTable.traceName;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.MSG_WRITER;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPERATION;
 
@@ -263,6 +273,9 @@ public class GridNioServer<T> {
      */
     private final boolean readWriteSelectorsAssign;
 
+    /** Tracing processor. */
+    private Tracing tracing;
+
     /**
      * @param addr Address.
      * @param port Port.
@@ -314,6 +327,7 @@ public class GridNioServer<T> {
         boolean readWriteSelectorsAssign,
         @Nullable GridWorkerListener workerLsnr,
         @Nullable MetricRegistry mreg,
+        Tracing tracing,
         GridNioFilter... filters
     ) throws IgniteCheckedException {
         if (port != -1)
@@ -340,6 +354,7 @@ public class GridNioServer<T> {
         this.selectorSpins = selectorSpins;
         this.readWriteSelectorsAssign = readWriteSelectorsAssign;
         this.lsnr = lsnr;
+        this.tracing = tracing == null ? new NoopTracing() : tracing;
 
         filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
 
@@ -1213,15 +1228,19 @@ public class GridNioServer<T> {
                 }
 
                 if (!skipWrite) {
-                    int cnt = sockCh.write(buf);
+                    Span span = tracing.create(COMMUNICATION_SOCKET_WRITE, req.span());
 
-                    if (log.isTraceEnabled())
-                        log.trace("Bytes sent [sockCh=" + sockCh + ", cnt=" + cnt + ']');
+                    try (TraceSurroundings ignore = span.equals(NoopSpan.INSTANCE) ? null : MTC.support(span)) {
+                        int cnt = sockCh.write(buf);
 
-                    if (sentBytesCntMetric != null)
-                        sentBytesCntMetric.add(cnt);
+                        if (log.isTraceEnabled())
+                            log.trace("Bytes sent [sockCh=" + sockCh + ", cnt=" + cnt + ']');
 
-                    ses.bytesSent(cnt);
+                        if (sentBytesCntMetric != null)
+                            sentBytesCntMetric.add(cnt);
+
+                        ses.bytesSent(cnt);
+                    }
                 }
                 else {
                     // For test purposes only (skipWrite is set to true in tests only).
@@ -1445,23 +1464,8 @@ public class GridNioServer<T> {
 
                     List<SessionWriteRequest> pendingRequests = new ArrayList<>(2);
 
-                    if (req != null) {
-                        msg = (Message)req.message();
-
-                        assert msg != null;
-
-                        if (writer != null)
-                            writer.setCurrentWriteClass(msg.getClass());
-
-                        finished = msg.writeTo(buf, writer);
-
-                        if (finished) {
-                            pendingRequests.add(req);
-
-                            if (writer != null)
-                                writer.reset();
-                        }
-                    }
+                    if (req != null)
+                        finished = writeToBuffer(writer, buf, req, pendingRequests);
 
                     // Fill up as many messages as possible to write buffer.
                     while (finished) {
@@ -1473,21 +1477,7 @@ public class GridNioServer<T> {
                         if (req == null)
                             break;
 
-                        msg = (Message)req.message();
-
-                        assert msg != null;
-
-                        if (writer != null)
-                            writer.setCurrentWriteClass(msg.getClass());
-
-                        finished = msg.writeTo(buf, writer);
-
-                        if (finished) {
-                            pendingRequests.add(req);
-
-                            if (writer != null)
-                                writer.reset();
-                        }
+                        finished = writeToBuffer(writer, buf, req, pendingRequests);
                     }
 
                     int sesBufLimit = buf.limit();
@@ -1557,6 +1547,46 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param writer Customizer of writing.
+         * @param buf Buffer to write.
+         * @param req Source of data.
+         * @param pendingRequests List of requests which was successfully written.
+         * @return {@code true} if message successfully written to buffer and {@code false} otherwise.
+         */
+        private boolean writeToBuffer(
+            MessageWriter writer,
+            ByteBuffer buf,
+            SessionWriteRequest req,
+            List<SessionWriteRequest> pendingRequests
+        ) {
+            Message msg;
+            boolean finished;
+            msg = (Message)req.message();
+
+            Span span = tracing.create(SpanType.COMMUNICATION_SOCKET_WRITE, req.span());
+
+            try (TraceSurroundings ignore = span.equals(NoopSpan.INSTANCE) ? null : MTC.support(span)) {
+                MTC.span().addTag(SpanTags.MESSAGE, () -> traceName(msg));
+
+                assert msg != null;
+
+                if (writer != null)
+                    writer.setCurrentWriteClass(msg.getClass());
+
+                finished = msg.writeTo(buf, writer);
+
+                if (finished) {
+                    pendingRequests.add(req);
+
+                    if (writer != null)
+                        writer.reset();
+                }
+
+                return finished;
+            }
+        }
+
+        /**
          * @param ses NIO session.
          * @param sockCh Socket channel.
          * @throws IOException If failed.
@@ -1644,26 +1674,10 @@ public class GridNioServer<T> {
                 }
             }
 
-            Message msg;
             boolean finished = false;
 
-            if (req != null) {
-                msg = (Message)req.message();
-
-                assert msg != null : req;
-
-                if (writer != null)
-                    writer.setCurrentWriteClass(msg.getClass());
-
-                finished = msg.writeTo(buf, writer);
-
-                if (finished) {
-                    onMessageWritten(ses, msg);
-
-                    if (writer != null)
-                        writer.reset();
-                }
-            }
+            if (req != null)
+                finished = writeToBuffer(ses, buf, req, writer);
 
             // Fill up as many messages as possible to write buffer.
             while (finished) {
@@ -1677,21 +1691,7 @@ public class GridNioServer<T> {
                 if (req == null)
                     break;
 
-                msg = (Message)req.message();
-
-                assert msg != null;
-
-                if (writer != null)
-                    writer.setCurrentWriteClass(msg.getClass());
-
-                finished = msg.writeTo(buf, writer);
-
-                if (finished) {
-                    onMessageWritten(ses, msg);
-
-                    if (writer != null)
-                        writer.reset();
-                }
+                finished = writeToBuffer(ses, buf, req, writer);
             }
 
             buf.flip();
@@ -1729,6 +1729,42 @@ public class GridNioServer<T> {
                 buf.clear();
         }
 
+        /**
+         * @param writer Customizer of writing.
+         * @param buf Buffer to write.
+         * @param req Source of data.
+         * @param ses Session for notification about writting.
+         * @return {@code true} if message successfully written to buffer and {@code false} otherwise.
+         */
+        private boolean writeToBuffer(GridSelectorNioSessionImpl ses, ByteBuffer buf, SessionWriteRequest req,
+            MessageWriter writer) {
+            Message msg;
+            boolean finished;
+            msg = (Message)req.message();
+
+            assert msg != null : req;
+
+            Span span = tracing.create(SpanType.COMMUNICATION_SOCKET_WRITE, req.span());
+
+            try (TraceSurroundings ignore = span.equals(NoopSpan.INSTANCE) ? null : MTC.support(span)) {
+                MTC.span().addTag(SpanTags.MESSAGE, () -> traceName(msg));
+
+                if (writer != null)
+                    writer.setCurrentWriteClass(msg.getClass());
+
+                finished = msg.writeTo(buf, writer);
+
+                if (finished) {
+                    onMessageWritten(ses, msg);
+
+                    if (writer != null)
+                        writer.reset();
+                }
+
+                return finished;
+            }
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(DirectNioClientWorker.class, this, super.toString());
@@ -2266,7 +2302,7 @@ public class GridNioServer<T> {
          * @param keys Keys.
          */
         private void dumpSelectorInfo(StringBuilder sb, Set<SelectionKey> keys) {
-            sb.append(">> Selector info [idx=").append(idx)
+            sb.append(">> Selector info [id=").append(idx)
                 .append(", keysCnt=").append(keys.size())
                 .append(", bytesRcvd=").append(bytesRcvd)
                 .append(", bytesRcvd0=").append(bytesRcvd0)
@@ -3138,6 +3174,9 @@ public class GridNioServer<T> {
         /** */
         private final GridNioSession ses;
 
+        /** */
+        private Span span;
+
         /**
          * @param ses Session.
          * @param msg Message.
@@ -3145,6 +3184,7 @@ public class GridNioServer<T> {
         WriteRequestSystemImpl(GridNioSession ses, Object msg) {
             this.ses = ses;
             this.msg = msg;
+            this.span = MTC.span();
         }
 
         /** {@inheritDoc} */
@@ -3203,6 +3243,11 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public Span span() {
+            return span;
+        }
+
+        /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(WriteRequestSystemImpl.class, this);
         }
@@ -3227,6 +3272,9 @@ public class GridNioServer<T> {
         /** */
         private final IgniteInClosure<IgniteException> ackC;
 
+        /** Span for tracing. */
+        private Span span;
+
         /**
          * @param ses Session.
          * @param msg Message.
@@ -3241,6 +3289,7 @@ public class GridNioServer<T> {
             this.msg = msg;
             this.skipRecovery = skipRecovery;
             this.ackC = ackC;
+            this.span = MTC.span();
         }
 
         /** {@inheritDoc} */
@@ -3301,6 +3350,11 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public Span span() {
+            return span;
+        }
+
+        /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(WriteRequestImpl.class, this);
         }
@@ -3337,6 +3391,9 @@ public class GridNioServer<T> {
         @GridToStringExclude
         private boolean skipRecovery;
 
+        /** */
+        private Span span;
+
         /**
          * @param sockCh Socket channel.
          * @param accepted {@code True} if socket has been accepted.
@@ -3354,6 +3411,7 @@ public class GridNioServer<T> {
             this.sockCh = sockCh;
             this.accepted = accepted;
             this.meta = meta;
+            this.span = MTC.span();
         }
 
         /**
@@ -3371,6 +3429,7 @@ public class GridNioServer<T> {
 
             this.ses = ses;
             this.op = op;
+            this.span = MTC.span();
         }
 
         /**
@@ -3395,6 +3454,7 @@ public class GridNioServer<T> {
             this.ses = ses;
             this.op = op;
             this.msg = msg;
+            this.span = MTC.span();
         }
 
         /**
@@ -3422,6 +3482,7 @@ public class GridNioServer<T> {
             this.op = op;
             this.msg = commMsg;
             this.skipRecovery = skipRecovery;
+            this.span = MTC.span();
         }
 
         /** {@inheritDoc} */
@@ -3435,6 +3496,11 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public Span span() {
+            return span;
+        }
+
+        /** {@inheritDoc} */
         @Override public Object message() {
             return msg;
         }
@@ -3730,6 +3796,9 @@ public class GridNioServer<T> {
         /** Metrics registry. */
         private MetricRegistry mreg;
 
+        /** Tracing processor */
+        private Tracing tracing;
+
         /**
          * Finishes building the instance.
          *
@@ -3760,6 +3829,7 @@ public class GridNioServer<T> {
                 readWriteSelectorsAssign,
                 workerLsnr,
                 mreg,
+                tracing,
                 filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
             );
 
@@ -3783,6 +3853,16 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param tracing Tracing processor.
+         * @return This for chaining.
+         */
+        public Builder<T> tracing(Tracing tracing) {
+            this.tracing = tracing;
+
+            return this;
+        }
+
+        /**
          * @param addr Local address.
          * @return This for chaining.
          */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioTracerFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioTracerFilter.java
new file mode 100644
index 0000000..31c69ef
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioTracerFilter.java
@@ -0,0 +1,125 @@
+/*
+ * 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.util.nio;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.tracing.MTC;
+import org.apache.ignite.internal.processors.tracing.NoopSpan;
+import org.apache.ignite.internal.processors.tracing.NoopTracing;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.processors.tracing.Tracing;
+import org.apache.ignite.internal.processors.tracing.messages.SpanTransport;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteInClosure;
+
+import static org.apache.ignite.internal.processors.tracing.SpanType.COMMUNICATION_SOCKET_READ;
+
+/**
+ * Filter that inject and extract tracing span from/to process.
+ */
+public class GridNioTracerFilter extends GridNioFilterAdapter {
+    /** Grid logger. */
+    @GridToStringExclude
+    private IgniteLogger log;
+
+    /** Tracing processor. */
+    private final Tracing tracer;
+
+    /**
+     * Creates a tracer filter.
+     *
+     * @param log Log instance to use.
+     * @param tracer Tracing processor.
+     */
+    public GridNioTracerFilter(IgniteLogger log, Tracing tracer) {
+        super("GridNioTracerFilter");
+
+        this.log = log;
+        this.tracer = tracer == null ? new NoopTracing() : tracer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridNioTracerFilter.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionOpened(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionClosed(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onExceptionCaught(
+        GridNioSession ses,
+        IgniteCheckedException ex
+    ) throws IgniteCheckedException {
+        proceedExceptionCaught(ses, ex);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
+    ) throws IgniteCheckedException {
+        if (msg instanceof SpanTransport && MTC.span() != NoopSpan.INSTANCE)
+            ((SpanTransport)msg).span(tracer.serialize(MTC.span()));
+
+        return proceedSessionWrite(ses, msg, fut, ackC);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        byte[] serializedSpan = msg instanceof SpanTransport ? ((SpanTransport)msg).span() : null;
+
+        if (serializedSpan != null && serializedSpan.length != 0) {
+            Span span = tracer.create(COMMUNICATION_SOCKET_READ, serializedSpan);
+
+            try (MTC.TraceSurroundings ignore = MTC.support(span)) {
+                proceedMessageReceived(ses, msg);
+            }
+        }
+        else
+            proceedMessageReceived(ses, msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) throws IgniteCheckedException {
+        return proceedSessionClose(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionIdleTimeout(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionWriteTimeout(ses);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index 87ecb30..7b8e6c0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -30,12 +30,14 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.processors.metric.impl.LongAdderMetric;
+import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.util.deque.FastSizeDeque;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.tracing.messages.TraceableMessagesTable.traceName;
 import static org.apache.ignite.internal.util.nio.GridNioServer.OUTBOUND_MESSAGES_QUEUE_SIZE_METRIC_DESC;
 import static org.apache.ignite.internal.util.nio.GridNioServer.OUTBOUND_MESSAGES_QUEUE_SIZE_METRIC_NAME;
 
@@ -307,6 +309,8 @@ public class GridSelectorNioSessionImpl extends GridNioSessionImpl implements Gr
 
         boolean res = queue.offerFirst(writeFut);
 
+        MTC.span().addLog(() -> "Added to system queue - " + traceName(writeFut.message()));
+
         assert res : "Future was not added to queue";
 
         if (outboundMessagesQueueSizeMetric != null)
@@ -335,6 +339,8 @@ public class GridSelectorNioSessionImpl extends GridNioSessionImpl implements Gr
 
         boolean res = queue.offer(writeFut);
 
+        MTC.span().addLog(() -> "Added to queue - " + traceName(writeFut.message()));
+
         assert res : "Future was not added to queue";
 
         if (outboundMessagesQueueSizeMetric != null)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
index e24f3ba..f9af37b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.nio;
 
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.tracing.Span;
 import org.apache.ignite.lang.IgniteInClosure;
 
 /**
@@ -75,4 +76,11 @@ public interface SessionWriteRequest {
      *
      */
     public void onMessageWritten();
+
+    /**
+     * Stored span for tracing.
+     *
+     * @return Span.
+     */
+    Span span();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tracing/configuration/VisorTracingConfigurationItem.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tracing/configuration/VisorTracingConfigurationItem.java
new file mode 100644
index 0000000..7ff0830
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tracing/configuration/VisorTracingConfigurationItem.java
@@ -0,0 +1,177 @@
+/*
+ * 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.visor.tracing.configuration;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Set;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.apache.ignite.internal.processors.tracing.Span;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.tracing.Scope;
+
+/**
+ * Data transfer object that contains scope, label, sampling rate and set of included scopes.
+ */
+@SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+public class VisorTracingConfigurationItem extends IgniteDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Specifies the {@link Scope} of a trace's root span to which some specific tracing configuration will be applied.
+     * It's a mandatory attribute.
+     */
+    private Scope scope;
+
+    /**
+     * Specifies the label of a traced operation. It's an optional attribute.
+     */
+    private String lb;
+
+    /**
+     * Number between 0 and 1 that more or less reflects the probability of sampling specific trace. 0 and 1 have
+     * special meaning here, 0 means never 1 means always. Default value is 0 (never).
+     */
+    private Double samplingRate;
+
+    /**
+     * Set of {@link Scope} that defines which sub-traces will be included in given trace. In other words, if child's
+     * span scope is equals to parent's scope or it belongs to the parent's span included scopes, then given child span
+     * will be attached to the current trace, otherwise it'll be skipped. See {@link
+     * Span#isChainable(Scope)} for more details.
+     */
+    private Set<Scope> includedScopes;
+
+    /**
+     * Default constructor.
+     */
+    public VisorTracingConfigurationItem() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param scope Specifies the {@link Scope} of a trace's root span to which some specific tracing configuration will be applied.
+     * @param lb Specifies the label of a traced operation. It's an optional attribute.
+     * @param samplingRate Number between 0 and 1 that more or less reflects the probability of sampling specific trace.
+     *  0 and 1 have special meaning here, 0 means never 1 means always. Default value is 0 (never).
+     * @param includedScopes Set of {@link Scope} that defines which sub-traces will be included in given trace.
+     *  In other words, if child's span scope is equals to parent's scope
+     *  or it belongs to the parent's span included scopes, then given child span will be attached to the current trace,
+     *  otherwise it'll be skipped.
+     *  See {@link Span#isChainable(Scope)} for more details.
+     */
+    public VisorTracingConfigurationItem(
+        Scope scope,
+        String lb,
+        Double samplingRate,
+        Set<Scope> includedScopes)
+    {
+        this.scope = scope;
+        this.lb = lb;
+        this.samplingRate = samplingRate;
+        this.includedScopes = includedScopes;
+    }
+
+    /**
+     * @return Specifies the  of a trace's root span to which some specific tracing configuration will be applied. It's
+     * a mandatory attribute.
+     */
+    public Scope scope() {
+        return scope;
+    }
+
+    /**
+     * @return Specifies the label of a traced operation. It's an optional attribute.
+     */
+    public String label() {
+        return lb;
+    }
+
+    /**
+     * @return Number between 0 and 1 that more or less reflects the probability of sampling specific trace. 0 and 1
+     * have special meaning here, 0 means never 1 means always. Default value is 0 (never).
+     */
+    public Double samplingRate() {
+        return samplingRate;
+    }
+
+    /**
+     * @return Set of  that defines which sub-traces will be included in given trace. In other words, if child's span
+     * scope is equals to parent's scope or it belongs to the parent's span included scopes, then given child span will
+     * be attached to the current trace, otherwise it'll be skipped. See  for more details.
+     */
+    public Set<Scope> includedScopes() {
+        return includedScopes;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
... 7386 lines suppressed ...