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 2019/12/26 09:24:42 UTC

[ignite] branch master updated: IGNITE-12440 Sensitive data leak in partition release future messages - Fixes #7195.

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 7af9887  IGNITE-12440 Sensitive data leak in partition release future messages - Fixes #7195.
7af9887 is described below

commit 7af988791dfd77de5679b87aac31ce99d45fcd05
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Thu Dec 26 12:05:59 2019 +0300

    IGNITE-12440 Sensitive data leak in partition release future messages - Fixes #7195.
    
    Signed-off-by: Ivan Rakov <iv...@gmail.com>
---
 .../org/apache/ignite/internal/IgniteKernal.java   |   4 +-
 .../internal/binary/BinaryClassDescriptor.java     |   4 +-
 .../internal/binary/BinaryEnumObjectImpl.java      |   2 +-
 .../internal/binary/BinaryFieldAccessor.java       |   4 +-
 .../ignite/internal/binary/BinaryObjectExImpl.java |   4 +-
 .../ignite/internal/binary/BinaryReaderExImpl.java |   2 +-
 .../managers/checkpoint/GridCheckpointManager.java |   4 +-
 .../processors/cache/CacheObjectAdapter.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java        |   4 +-
 .../processors/cache/KeyCacheObjectImpl.java       |   2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java    |   4 +-
 .../cache/store/GridCacheStoreManagerAdapter.java  |   2 +-
 .../cache/store/GridCacheWriteBehindStore.java     |   4 +-
 .../processors/cache/transactions/IgniteTxKey.java |   2 +-
 .../processors/cache/transactions/TxDeadlock.java  |   2 +-
 .../cache/transactions/TxEntryValueHolder.java     |   8 +-
 .../platform/PlatformNativeException.java          |   2 +-
 .../util/tostring/GridToStringBuilder.java         |  64 +++-
 .../GridCacheBinaryObjectsAbstractSelfTest.java    |   2 +-
 .../util/tostring/GridToStringBuilderSelfTest.java |   4 +-
 .../tostring/TransactionSensitiveDataTest.java     | 343 +++++++++++++++++++++
 .../testframework/junits/GridAbstractTest.java     |  24 +-
 .../ignite/testsuites/IgniteUtilSelfTestSuite.java |   2 +
 .../processors/query/h2/dml/UpdatePlanBuilder.java |   4 +-
 .../processors/query/h2/opt/H2CacheRow.java        |   4 +-
 .../index/H2TreeCorruptedTreeExceptionTest.java    |  74 ++---
 26 files changed, 493 insertions(+), 84 deletions(-)

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 067ff4e..65f2c90 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
@@ -2780,7 +2780,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     private void ackSystemProperties() {
         assert log != null;
 
-        if (log.isDebugEnabled() && S.INCLUDE_SENSITIVE)
+        if (log.isDebugEnabled() && S.includeSensitive())
             for (Map.Entry<Object, Object> entry : snapshot().entrySet())
                 log.debug("System property [" + entry.getKey() + '=' + entry.getValue() + ']');
     }
@@ -3003,7 +3003,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         assert log != null;
 
         // Ack IGNITE_HOME and VM arguments.
-        if (log.isInfoEnabled() && S.INCLUDE_SENSITIVE) {
+        if (log.isInfoEnabled() && S.includeSensitive()) {
             log.info("IGNITE_HOME=" + cfg.getIgniteHome());
             log.info("VM arguments: " + rtBean.getInputArguments());
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index b4fd932..4ae17cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -829,7 +829,7 @@ public class BinaryClassDescriptor {
         catch (Exception e) {
             String msg;
 
-            if (S.INCLUDE_SENSITIVE && !F.isEmpty(typeName))
+            if (S.includeSensitive() && !F.isEmpty(typeName))
                 msg = "Failed to serialize object [typeName=" + typeName + ']';
             else
                 msg = "Failed to serialize object [typeId=" + typeId + ']';
@@ -903,7 +903,7 @@ public class BinaryClassDescriptor {
         catch (Exception e) {
             String msg;
 
-            if (S.INCLUDE_SENSITIVE && !F.isEmpty(typeName))
+            if (S.includeSensitive() && !F.isEmpty(typeName))
                 msg = "Failed to deserialize object [typeName=" + typeName + ']';
             else
                 msg = "Failed to deserialize object [typeId=" + typeId + ']';
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index dd99787..02bfe39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -230,7 +230,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        if (!S.INCLUDE_SENSITIVE)
+        if (!S.includeSensitive())
             return ord >= 0 ? "BinaryEnum" : "null";
 
         // 1. Try deserializing the object.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
index bd5ded5..4f159a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
@@ -160,7 +160,7 @@ public abstract class BinaryFieldAccessor {
             throw ex;
         }
         catch (Exception ex) {
-            if (S.INCLUDE_SENSITIVE && !F.isEmpty(name))
+            if (S.includeSensitive() && !F.isEmpty(name))
                 throw new BinaryObjectException("Failed to write field [name=" + name + ']', ex);
             else
                 throw new BinaryObjectException("Failed to write field [id=" + id + ']', ex);
@@ -188,7 +188,7 @@ public abstract class BinaryFieldAccessor {
             read0(obj, reader);
         }
         catch (Exception ex) {
-            if (S.INCLUDE_SENSITIVE && !F.isEmpty(name))
+            if (S.includeSensitive() && !F.isEmpty(name))
                 throw new BinaryObjectException("Failed to read field [name=" + name + ']', ex);
             else
                 throw new BinaryObjectException("Failed to read field [id=" + id + ']', ex);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
index 51599f8..e11d8e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
@@ -214,8 +214,8 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
             IgniteThread.onForbidBinaryMetadataRequestSectionLeft();
         }
 
-        if (meta == null || !S.INCLUDE_SENSITIVE)
-            return S.toString(S.INCLUDE_SENSITIVE ? BinaryObject.class.getSimpleName() : "BinaryObject",
+        if (meta == null || !S.includeSensitive())
+            return S.toString(S.includeSensitive() ? BinaryObject.class.getSimpleName() : "BinaryObject",
                 "idHash", idHash, false,
                 "hash", hash, false,
                 "typeId", typeId(), true);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
index 1319247..02d6d4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
@@ -443,7 +443,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @return wrapping exception
      */
     private BinaryObjectException wrapFieldException(String fieldName, Exception e) {
-        if (S.INCLUDE_SENSITIVE)
+        if (S.includeSensitive())
             return new BinaryObjectException("Failed to read field: " + fieldName, e);
         else
             return new BinaryObjectException("Failed to read field.", e);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
index b256dec..2404e01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
@@ -354,7 +354,7 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
             return state;
         }
         catch (IgniteSpiException e) {
-            throw new IgniteCheckedException(S.INCLUDE_SENSITIVE ?
+            throw new IgniteCheckedException(S.includeSensitive() ?
                 ("Failed to load checkpoint: " + key) : "Failed to load checkpoint", e);
         }
     }
@@ -407,7 +407,7 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
 
                 msg = "Checkpoint removed";
             }
-            if (S.INCLUDE_SENSITIVE)
+            if (S.includeSensitive())
                 msg += ": " + key;
 
             ctx.event().record(new CheckpointEvent(ctx.discovery().localNode(), msg, type, key));
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index c6d9002..fa51ce5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -173,7 +173,7 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(S.INCLUDE_SENSITIVE ? getClass().getSimpleName() : "CacheObject",
+        return S.toString(S.includeSensitive() ? getClass().getSimpleName() : "CacheObject",
             "val", val, true,
             "hasValBytes", valBytes != null, false);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index b55c121..5969666 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -206,11 +206,11 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     protected final GridCacheContext<?, ?> cctx;
 
     /** Key. */
-    @GridToStringInclude
+    @GridToStringInclude(sensitive = true)
     protected final KeyCacheObject key;
 
     /** Value. */
-    @GridToStringInclude
+    @GridToStringInclude(sensitive = true)
     protected CacheObject val;
 
     /** Version. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
index 89e4338..553c6aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
@@ -205,7 +205,7 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(S.INCLUDE_SENSITIVE ? getClass().getSimpleName() : "KeyCacheObject",
+        return S.toString(S.includeSensitive() ? getClass().getSimpleName() : "KeyCacheObject",
             "part", part, true,
             "val", val, true,
             "hasValBytes", valBytes != null, false);
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 78fbe9f..0024e11 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
@@ -1233,7 +1233,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
 
             assert key != null : entry.key();
 
-            if (S.INCLUDE_SENSITIVE)
+            if (S.includeSensitive())
                 msg.append("key=").append(key.toString()).append(", keyCls=").append(key.getClass().getName());
         }
         catch (Exception e) {
@@ -1248,7 +1248,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
             Object val = cacheVal != null ? cctx.unwrapBinaryIfNeeded(cacheVal, entry.keepBinary(), false) : null;
 
             if (val != null) {
-                if (S.INCLUDE_SENSITIVE)
+                if (S.includeSensitive())
                     msg.append(", val=").append(val.toString()).append(", valCls=").append(val.getClass().getName());
             }
             else
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 2f7b02b..414fdc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -1340,7 +1340,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            if (!S.INCLUDE_SENSITIVE)
+            if (!S.includeSensitive())
                 return "[size=" + size() + "]";
 
             Iterator<Cache.Entry<?, ?>> it = iterator();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
index 565ecd6..02beb1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
@@ -54,7 +54,7 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentLinkedHashMap;
 
 import static javax.cache.Cache.Entry;
-import static org.apache.ignite.internal.util.tostring.GridToStringBuilder.INCLUDE_SENSITIVE;
+import static org.apache.ignite.internal.util.tostring.GridToStringBuilder.includeSensitive;
 
 /**
  * Internal wrapper for a {@link CacheStore} that enables write-behind logic.
@@ -906,7 +906,7 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
 
                     log.error("Failed to update store (value will be lost as current buffer size is greater " +
                         "than 'cacheCriticalSize' or node has been stopped before store was repaired) [" +
-                        (INCLUDE_SENSITIVE ? "key=" + entry.getKey() + ", val=" + val + ", " : "") +
+                        (includeSensitive() ? "key=" + entry.getKey() + ", val=" + val + ", " : "") +
                         "op=" + operation + "]");
                 }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
index 2c3892f..76c23a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
@@ -37,7 +37,7 @@ public class IgniteTxKey implements Message {
     private static final long serialVersionUID = 0L;
 
     /** Key. */
-    @GridToStringInclude
+    @GridToStringInclude(sensitive = true)
     private KeyCacheObject key;
 
     /** Cache ID. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java
index 54fc9b3..d514824 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java
@@ -140,7 +140,7 @@ public class TxDeadlock {
 
                 sb.append(e.getValue())
                     .append(" [");
-                if (S.INCLUDE_SENSITIVE)
+                if (S.includeSensitive())
                     sb.append("key=")
                         .append(val)
                         .append(", ");
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
index a0d9058..051a11c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
@@ -24,7 +24,9 @@ import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 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;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -45,7 +47,7 @@ public class TxEntryValueHolder implements Message {
     private static final long serialVersionUID = 0L;
 
     /** */
-    @GridToStringInclude
+    @GridToStringInclude(sensitive = true)
     private CacheObject val;
 
     /** */
@@ -53,9 +55,11 @@ public class TxEntryValueHolder implements Message {
     private GridCacheOperation op = NOOP;
 
     /** Flag indicating that value has been set for write. */
+    @GridToStringExclude
     private boolean hasWriteVal;
 
     /** Flag indicating that value has been set for read. */
+    @GridToStringExclude
     @GridDirectTransient
     private boolean hasReadVal;
 
@@ -158,7 +162,7 @@ public class TxEntryValueHolder implements Message {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return "[op=" + op +", val=" + val + ']';
+        return S.toString(TxEntryValueHolder.class, this);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
index 6c65c56..b87c2f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
@@ -72,6 +72,6 @@ public class PlatformNativeException extends PlatformException implements Extern
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(PlatformNativeException.class, this,
-            "cause", S.INCLUDE_SENSITIVE ? cause : (cause == null ? "null" : cause.getClass().getSimpleName()));
+            "cause", S.includeSensitive() ? cause : (cause == null ? "null" : cause.getClass().getSimpleName()));
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
index be42f2e..58abd55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
@@ -33,9 +33,11 @@ import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
+import java.util.function.Supplier;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.util.GridUnsafe;
@@ -43,8 +45,10 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static java.util.Objects.nonNull;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_COLLECTION_LIMIT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 
 /**
  * Provides auto-generation framework for {@code toString()} output.
@@ -91,9 +95,17 @@ public class GridToStringBuilder {
     /** */
     private static final Map<String, GridToStringClassDescriptor> classCache = new ConcurrentHashMap<>();
 
-    /** {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE} */
-    public static final boolean INCLUDE_SENSITIVE =
-        IgniteSystemProperties.getBoolean(IGNITE_TO_STRING_INCLUDE_SENSITIVE, true);
+    /** Supplier for {@link #includeSensitive} with default behavior. */
+    private static final AtomicReference<Supplier<Boolean>> INCL_SENS_SUP_REF =
+        new AtomicReference<>(new Supplier<Boolean>() {
+            /** Value of "IGNITE_TO_STRING_INCLUDE_SENSITIVE". */
+            final boolean INCLUDE_SENSITIVE = getBoolean(IGNITE_TO_STRING_INCLUDE_SENSITIVE, true);
+
+            /** {@inheritDoc} */
+            @Override public Boolean get() {
+                return INCLUDE_SENSITIVE;
+            }
+        });
 
     /** */
     private static final int COLLECTION_LIMIT =
@@ -124,6 +136,44 @@ public class GridToStringBuilder {
     };
 
     /**
+     * Implementation of the <a href=
+     * "https://en.wikipedia.org/wiki/Initialization-on-demand_holder_idiom">
+     * "Initialization-on-demand holder idiom"</a>.
+     */
+    private static class Holder {
+        /** Supplier holder for {@link #includeSensitive}. */
+        static final Supplier<Boolean> INCL_SENS_SUP = INCL_SENS_SUP_REF.get();
+    }
+
+    /**
+     * Setting the logic of the {@link #includeSensitive} method. <br/>
+     * By default, it take the value of
+     * {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE
+     * IGNITE_TO_STRING_INCLUDE_SENSITIVE} system property. <br/>
+     * <b>Important!</b> Changing the logic is possible only until the first
+     * call of  {@link #includeSensitive} method. <br/>
+     *
+     * @param sup
+     */
+    public static void setIncludeSensitiveSupplier(Supplier<Boolean> sup) {
+        assert nonNull(sup);
+
+        INCL_SENS_SUP_REF.set(sup);
+    }
+
+    /**
+     * Return {@code true} if need to include sensitive data otherwise
+     * {@code false}.
+     *
+     * @return {@code true} if need to include sensitive data otherwise
+     *      {@code false}.
+     * @see GridToStringBuilder#setIncludeSensitiveSupplier(Supplier)
+     */
+    public static boolean includeSensitive() {
+        return Holder.INCL_SENS_SUP.get();
+    }
+
+    /**
      * @param obj Object.
      * @return Hexed identity hashcode.
      */
@@ -1187,7 +1237,7 @@ public class GridToStringBuilder {
 
         for (int i = 0; i <= idxMax; ++i) {
             b.append(Array.get(arr, i));
-            
+
             if (i == idxMax)
                 return b.append(']').toString();
 
@@ -1671,12 +1721,12 @@ public class GridToStringBuilder {
                 Object addVal = addVals[i];
 
                 if (addVal != null) {
-                    if (addSens != null && addSens[i] && !INCLUDE_SENSITIVE)
+                    if (addSens != null && addSens[i] && !includeSensitive())
                         continue;
 
                     GridToStringInclude incAnn = addVal.getClass().getAnnotation(GridToStringInclude.class);
 
-                    if (incAnn != null && incAnn.sensitive() && !INCLUDE_SENSITIVE)
+                    if (incAnn != null && incAnn.sensitive() && !includeSensitive())
                         continue;
                 }
 
@@ -1722,7 +1772,7 @@ public class GridToStringBuilder {
                     // Information is not sensitive when both the field and the field type are not sensitive.
                     // When @GridToStringInclude is not present then the flag is false by default for that attribute.
                     final boolean notSens = (incFld == null || !incFld.sensitive()) && (incType == null || !incType.sensitive());
-                    add = notSens || INCLUDE_SENSITIVE;
+                    add = notSens || includeSensitive();
                 }
                 else if (!f.isAnnotationPresent(GridToStringExclude.class) &&
                     !type.isAnnotationPresent(GridToStringExclude.class)
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 7a09072..484658c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -238,7 +238,7 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
         String typeName = nameMapper.typeName(TestReferenceObject.class.getName());
 
         assertTrue("Unexpected toString: " + str,
-            S.INCLUDE_SENSITIVE ?
+            S.includeSensitive() ?
             str.startsWith(typeName) && str.contains("obj=" + typeName + " [") :
             str.startsWith("BinaryObject") && str.contains("idHash=") && str.contains("hash=")
         );
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java
index 197d346..bd57a6d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java
@@ -671,7 +671,7 @@ public class GridToStringBuilderSelfTest extends GridCommonAbstractTest {
             buf.append("id=").append(id).append(", ");
             buf.append("uuidVar=").append(uuidVar).append(", ");
             buf.append("intVar=").append(intVar).append(", ");
-            if (S.INCLUDE_SENSITIVE)
+            if (S.includeSensitive())
                 buf.append("longVar=").append(longVar).append(", ");
             buf.append("boolVar=").append(boolVar).append(", ");
             buf.append("byteVar=").append(byteVar).append(", ");
@@ -706,7 +706,7 @@ public class GridToStringBuilderSelfTest extends GridCommonAbstractTest {
             StringBuilder s = new StringBuilder(toStringManual());
             s.setLength(s.length() - 1);
             s.append(", newParam1=").append(1);
-            if (S.INCLUDE_SENSITIVE)
+            if (S.includeSensitive())
                 s.append(", newParam2=").append(2);
             s.append(']');
             return s.toString();
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/TransactionSensitiveDataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/TransactionSensitiveDataTest.java
new file mode 100644
index 0000000..993821b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/TransactionSensitiveDataTest.java
@@ -0,0 +1,343 @@
+/*
+ * 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.tostring;
+
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFutureAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static java.lang.Thread.currentThread;
+import static java.util.Objects.nonNull;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.testframework.GridTestUtils.assertContains;
+import static org.apache.ignite.testframework.GridTestUtils.assertNotContains;
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValue;
+import static org.apache.ignite.testframework.GridTestUtils.setFieldValue;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Class for checking sensitive data when outputting transactions to the log.
+ */
+public class TransactionSensitiveDataTest extends GridCommonAbstractTest {
+    /** Listener log messages. */
+    private static ListeningTestLogger testLog;
+
+    /** Node count. */
+    private static final int NODE_COUNT = 2;
+
+    /** Create a client node. */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        setFieldValue(GridNearTxPrepareFutureAdapter.class, "log", null);
+        ((AtomicReference<IgniteLogger>)getFieldValue(GridNearTxPrepareFutureAdapter.class, "logRef")).set(null);
+
+        clearGridToStringClassCache();
+
+        testLog = new ListeningTestLogger(false, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        testLog.clearListeners();
+
+        stopAllGrids();
+
+        clearGridToStringClassCache();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setConsistentId(igniteInstanceName)
+            .setGridLogger(testLog)
+            .setClientMode(client)
+            .setCommunicationSpi(new TestRecordingCommunicationSpi())
+            .setCacheConfiguration(
+                new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+                    .setAtomicityMode(TRANSACTIONAL)
+                    .setBackups(NODE_COUNT)
+                    .setAffinity(new RendezvousAffinityFunction(false, 10))
+            );
+    }
+
+    /**
+     * Test for checking the absence of sensitive data in log during an
+     * exchange while an active transaction is running.
+     *
+     * @throws Exception If failed.
+     */
+    @WithSystemProperty(key = IGNITE_TO_STRING_INCLUDE_SENSITIVE, value = "false")
+    @Test
+    public void testHideSensitiveDataDuringExchange() throws Exception {
+        checkSensitiveDataDuringExchange((logStr, binObjStr) -> assertNotContains(log, logStr, binObjStr));
+    }
+
+    /**
+     * Test for checking the presence of sensitive data in log during an
+     * exchange while an active transaction is running.
+     *
+     * @throws Exception If failed.
+     */
+    @WithSystemProperty(key = IGNITE_TO_STRING_INCLUDE_SENSITIVE, value = "true")
+    @Test
+    public void testShowSensitiveDataDuringExchange() throws Exception {
+        checkSensitiveDataDuringExchange((logStr, binObjStr) -> assertContains(log, logStr, binObjStr));
+    }
+
+    /**
+     * Test for checking the absence of sensitive data in log when node exits
+     * during transaction preparation.
+     *
+     * @throws Exception If failed.
+     */
+    @WithSystemProperty(key = IGNITE_TO_STRING_INCLUDE_SENSITIVE, value = "false")
+    @Test
+    public void testHideSensitiveDataDuringNodeLeft() throws Exception {
+        checkSensitiveDataDuringNodeLeft((logStr, binObjStr) -> assertNotContains(log, logStr, binObjStr));
+    }
+
+    /**
+     * Test for checking the presence of sensitive data in log when node exits
+     * during transaction preparation.
+     *
+     * @throws Exception If failed.
+     */
+    @WithSystemProperty(key = IGNITE_TO_STRING_INCLUDE_SENSITIVE, value = "true")
+    @Test
+    public void testShowSensitiveDataDuringNodeLeft() throws Exception {
+        checkSensitiveDataDuringNodeLeft((logStr, binObjStr) -> assertContains(log, logStr, binObjStr));
+    }
+
+    /**
+     * Receiving a log message "Partition release future:" during the exchange
+     * to check whether or not sensitive data is in printed transactions.
+     *
+     * @param check Check sensitive data in log message.
+     * @throws Exception If failed.
+     */
+    private void checkSensitiveDataDuringExchange(BiConsumer<String, String> check) throws Exception {
+        assert nonNull(check);
+
+        IgniteEx crd = startGrids(NODE_COUNT);
+
+        awaitPartitionMapExchange();
+
+        AtomicReference<String> strToCheckRef = new AtomicReference<>();
+
+        LogListener logLsnr = LogListener.matches(logStr -> {
+            if (logStr.contains("Partition release future:") && currentThread().getName().contains(crd.name())) {
+                strToCheckRef.set(logStr);
+
+                return true;
+            }
+
+            return false;
+        }).build();
+
+        testLog.registerListener(logLsnr);
+
+        IgniteCache<Object, Object> cache = crd.getOrCreateCache(DEFAULT_CACHE_NAME).withKeepBinary();
+
+        IgniteBinary binary = crd.binary();
+
+        BinaryObject binKey = binary.toBinary(new Key(0));
+        BinaryObject binPerson = binary.toBinary(new Person(1, "name_1"));
+
+        cache.put(binKey, binPerson);
+
+        Transaction tx = crd.transactions().txStart();
+
+        cache.put(binKey, binPerson);
+
+        GridTestUtils.runAsync(() -> {
+            logLsnr.check(10 * crd.configuration().getNetworkTimeout());
+
+            tx.commit();
+
+            return null;
+        });
+
+        startGrid(NODE_COUNT);
+
+        check.accept(maskIdHash(strToCheckRef.get()), maskIdHash(toStr(binKey, Key.class)));
+        check.accept(maskIdHash(strToCheckRef.get()), maskIdHash(toStr(binPerson, Person.class)));
+    }
+
+    /**
+     * Receiving the “Failed to send message to remote node” and
+     * “Received error when future is done” message logs during the node exit
+     * when preparing the transaction to check whether or not sensitive data
+     * is in the printed transactions.
+     *
+     * @param check Check sensitive data in log messages.
+     * @throws Exception If failed.
+     */
+    private void checkSensitiveDataDuringNodeLeft(BiConsumer<String, String> check) throws Exception {
+        assert nonNull(check);
+
+        client = false;
+
+        startGrids(NODE_COUNT);
+
+        client = true;
+
+        IgniteEx clientNode = startGrid(NODE_COUNT);
+
+        awaitPartitionMapExchange();
+
+        AtomicReference<String> strFailedSndRef = new AtomicReference<>();
+        AtomicReference<String> strReceivedErrorRef = new AtomicReference<>();
+
+        testLog.registerListener(logStr -> {
+            if (logStr.contains("Failed to send message to remote node"))
+                strFailedSndRef.set(logStr);
+        });
+
+        testLog.registerListener(logStr -> {
+            if (logStr.contains("Received error when future is done"))
+                strReceivedErrorRef.set(logStr);
+        });
+
+        int stopGridId = 0;
+
+        TestRecordingCommunicationSpi.spi(clientNode).closure((clusterNode, message) -> {
+            if (GridNearTxPrepareRequest.class.isInstance(message))
+                stopGrid(stopGridId);
+        });
+
+        String cacheName = DEFAULT_CACHE_NAME;
+
+        IgniteCache<Object, Object> cache = clientNode.getOrCreateCache(cacheName).withKeepBinary();
+
+        IgniteBinary binary = clientNode.binary();
+
+        BinaryObject binKey = binary.toBinary(new Key(primaryKey(grid(stopGridId).cache(cacheName))));
+        BinaryObject binPerson = binary.toBinary(new Person(1, "name_1"));
+
+        try (Transaction tx = clientNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.put(binKey, binPerson);
+
+            tx.commit();
+        }
+        catch (Exception ignored) {
+            //ignore
+        }
+
+        String strFailedSndStr = maskIdHash(strFailedSndRef.get());
+        String strReceivedErrorStr = maskIdHash(strReceivedErrorRef.get());
+
+        String binKeyStr = maskIdHash(toStr(binKey, Key.class));
+        String binPersonStr = maskIdHash(toStr(binPerson, Person.class));
+
+        check.accept(strFailedSndStr, binKeyStr);
+        check.accept(strFailedSndStr, binPersonStr);
+
+        check.accept(strReceivedErrorStr, binKeyStr);
+        check.accept(strReceivedErrorStr, binPersonStr);
+    }
+
+    /**
+     * Removes a idHash from a string.
+     *
+     * @param s String.
+     * @return String without a idHash.
+     */
+    private String maskIdHash(String s) {
+        assert nonNull(s);
+
+        return s.replaceAll("idHash=[0-9]*", "idHash=NO");
+    }
+
+    /**
+     * Create a string to search for BinaryObject in the log.
+     *
+     * @param binPerson BinaryObject.
+     * @param cls Class of BinaryObject.
+     * @return String representation of BinaryObject.
+     */
+    private String toStr(BinaryObject binPerson, Class<?> cls) {
+        assert nonNull(binPerson);
+        assert nonNull(cls);
+
+        return binPerson.toString().replace(cls.getName(), cls.getSimpleName());
+    }
+
+    /**
+     * Key for mapping value in cache.
+     */
+    static class Key {
+        /** Id. */
+        int id;
+
+        /**
+         * Constructor.
+         *
+         * @param id Id.
+         */
+        public Key(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     * Person class for cache storage.
+     */
+    static class Person {
+        /** Id organization. */
+        int orgId;
+
+        /** Person name. */
+        String name;
+
+        /**
+         * Constructor.
+         *
+         * @param orgId Id organization.
+         * @param name Person name.
+         */
+        public Person(int orgId, String name) {
+            this.orgId = orgId;
+            this.name = name;
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 97e16fd..8173f43 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -56,7 +56,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.cluster.ClusterNode;
@@ -89,6 +88,7 @@ import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
@@ -143,13 +143,19 @@ import org.springframework.beans.BeansException;
 import org.springframework.context.ApplicationContext;
 import org.springframework.context.support.FileSystemXmlApplicationContext;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_ALLOW_ATOMIC_OPS_IN_TX;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CLIENT_CACHE_CHANGE_MESSAGE_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_LOG_CLASSPATH_CONTENT_ON_STARTUP;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValueHierarchy;
 import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
 import static org.apache.ignite.testframework.config.GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS;
 
@@ -247,13 +253,15 @@ public abstract class GridAbstractTest extends JUnitAssertAware {
 
     /** */
     static {
-        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_ATOMIC_OPS_IN_TX, "false");
-        System.setProperty(IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "10000");
-        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "false");
+        System.setProperty(IGNITE_ALLOW_ATOMIC_OPS_IN_TX, "false");
+        System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "10000");
+        System.setProperty(IGNITE_UPDATE_NOTIFIER, "false");
         System.setProperty(IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY, "1");
         System.setProperty(IGNITE_CLIENT_CACHE_CHANGE_MESSAGE_TIMEOUT, "1000");
         System.setProperty(IGNITE_LOG_CLASSPATH_CONTENT_ON_STARTUP, "false");
 
+        S.setIncludeSensitiveSupplier(() -> getBoolean(IGNITE_TO_STRING_INCLUDE_SENSITIVE, true));
+
         if (GridTestClockTimer.startTestTimer()) {
             Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests");
 
@@ -2386,6 +2394,14 @@ public abstract class GridAbstractTest extends JUnitAssertAware {
     }
 
     /**
+     * Clear S#classCache. Use if necessary to test sensitive data
+     * in the test. https://ggsystems.atlassian.net/browse/GG-25182
+     */
+    protected void clearGridToStringClassCache() {
+        ((Map)getFieldValueHierarchy(S.class, "classCache")).clear();
+    }
+
+    /**
      * @param millis Time to sleep.
      */
     public static void doSleep(long millis) {
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index 5ed43d9..ba8f3fe 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.util.tostring.CircularStringBuilderSelfTest;
 import org.apache.ignite.internal.util.tostring.GridToStringBuilderSelfTest;
 import org.apache.ignite.internal.util.tostring.IncludeSensitiveAtomicTest;
 import org.apache.ignite.internal.util.tostring.IncludeSensitiveTransactionalTest;
+import org.apache.ignite.internal.util.tostring.TransactionSensitiveDataTest;
 import org.apache.ignite.lang.GridByteArrayListSelfTest;
 import org.apache.ignite.spi.discovery.ClusterMetricsSelfTest;
 import org.apache.ignite.spi.discovery.ClusterMetricsSnapshotSerializeCompatibilityTest;
@@ -103,6 +104,7 @@ import org.junit.runners.Suite;
     // Sensitive toString.
     IncludeSensitiveAtomicTest.class,
     IncludeSensitiveTransactionalTest.class,
+    TransactionSensitiveDataTest.class,
 
     // Metrics.
     ClusterMetricsSnapshotSerializeSelfTest.class,
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index bd760c2..1d07e89 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -690,7 +690,7 @@ public final class UpdatePlanBuilder {
                             return ctor0.newInstance();
                         }
                         catch (Exception e) {
-                            if (S.INCLUDE_SENSITIVE)
+                            if (S.includeSensitive())
                                 throw new IgniteCheckedException("Failed to instantiate " +
                                     (key ? "key" : "value") + " [type=" + typeName + ']', e);
                             else
@@ -709,7 +709,7 @@ public final class UpdatePlanBuilder {
                             return GridUnsafe.allocateInstance(cls);
                         }
                         catch (InstantiationException e) {
-                            if (S.INCLUDE_SENSITIVE)
+                            if (S.includeSensitive())
                                 throw new IgniteCheckedException("Failed to instantiate " +
                                     (key ? "key" : "value") + " [type=" + typeName + ']', e);
                             else
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
index b9998a6..c302aa8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
@@ -313,7 +313,7 @@ public class H2CacheRow extends H2Row implements CacheDataRow {
         sb.a("[ key: ").a(v == null ? "nil" : v.getString());
 
         v = valueWrapped();
-        sb.a(", val: ").a(v == null ? "nil" : (S.INCLUDE_SENSITIVE ? v.getString() :
+        sb.a(", val: ").a(v == null ? "nil" : (S.includeSensitive() ? v.getString() :
             "Data hidden due to " + IGNITE_TO_STRING_INCLUDE_SENSITIVE + " flag."));
 
         sb.a(" ][ ");
@@ -326,7 +326,7 @@ public class H2CacheRow extends H2Row implements CacheDataRow {
                     sb.a(", ");
 
                 if (!desc.isKeyValueOrVersionColumn(i))
-                    sb.a(v == null ? "nil" : (S.INCLUDE_SENSITIVE ? v.getString() : "data hidden"));
+                    sb.a(v == null ? "nil" : (S.includeSensitive() ? v.getString() : "data hidden"));
             }
         }
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TreeCorruptedTreeExceptionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TreeCorruptedTreeExceptionTest.java
index 203fa7b..82bcce9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TreeCorruptedTreeExceptionTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TreeCorruptedTreeExceptionTest.java
@@ -33,11 +33,10 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHan
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandlerWrapper;
 import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
 import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.ListeningTestLogger;
 import org.apache.ignite.testframework.LogListener;
 import org.apache.ignite.testframework.MessageOrderLogListener;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
@@ -142,63 +141,58 @@ public class H2TreeCorruptedTreeExceptionTest extends GridCommonAbstractTest {
 
         cleanPersistenceDir();
 
+        clearGridToStringClassCache();
+
         super.afterTest();
     }
 
     /** */
     @Test
+    @WithSystemProperty(key = IGNITE_TO_STRING_INCLUDE_SENSITIVE, value = "false")
     public void testCorruptedTree() throws Exception {
-        boolean curSensVal = S.INCLUDE_SENSITIVE;
-
-        try {
-            IgniteEx srv = startGrid(0);
-
-            srv.cluster().active(true);
+        IgniteEx srv = startGrid(0);
 
-            GridTestUtils.setFieldValue(S.class.getField("INCLUDE_SENSITIVE"), false);
+        srv.cluster().active(true);
 
-            IgniteCache<Integer, Integer> cache = srv.getOrCreateCache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, Integer> cache = srv.getOrCreateCache(DEFAULT_CACHE_NAME);
 
-            cache.query(new SqlFieldsQuery("create table a (col1 varchar primary key, col2 varchar) with " +
-                "\"CACHE_GROUP=" + GRP_NAME + "\""));
-            cache.query(new SqlFieldsQuery("create index " + IDX_NAME + " on a(col2)"));
+        cache.query(new SqlFieldsQuery("create table a (col1 varchar primary key, col2 varchar) with " +
+            "\"CACHE_GROUP=" + GRP_NAME + "\""));
+        cache.query(new SqlFieldsQuery("create index " + IDX_NAME + " on a(col2)"));
 
-            failWithCorruptTree.set(true);
-
-            try {
-                cache.query(new SqlFieldsQuery("insert into a(col1, col2) values (1, ?1)")
-                    .setArgs(VERY_SENS_STR_DATA));
-
-                fail("Cache operations are expected to fail");
-            }
-            catch (Throwable e) {
-                assertTrue(X.hasCause(e, CorruptedTreeException.class));
-            }
+        failWithCorruptTree.set(true);
 
-            assertTrue(logListener.check());
+        try {
+            cache.query(new SqlFieldsQuery("insert into a(col1, col2) values (1, ?1)")
+                .setArgs(VERY_SENS_STR_DATA));
 
-            assertFalse(logSensListener.check());
+            fail("Cache operations are expected to fail");
+        }
+        catch (Throwable e) {
+            assertTrue(X.hasCause(e, CorruptedTreeException.class));
+        }
 
-            GridTestUtils.setFieldValue(S.class.getField("INCLUDE_SENSITIVE"), true);
+        assertTrue(logListener.check());
 
-            logListener.reset();
+        assertFalse(logSensListener.check());
 
-            logSensListener.reset();
+        System.setProperty(IGNITE_TO_STRING_INCLUDE_SENSITIVE, Boolean.TRUE.toString());
+        clearGridToStringClassCache();
 
-            try {
-                cache.query(new SqlFieldsQuery("insert into a(col1, col2) values (2, ?1)")
-                    .setArgs(VERY_SENS_STR_DATA));
-            }
-            catch (Throwable e) {
-                assertTrue(X.hasCause(e, CorruptedTreeException.class));
-            }
+        logListener.reset();
 
-            assertFalse(logListener.check());
+        logSensListener.reset();
 
-            assertTrue(logSensListener.check());
+        try {
+            cache.query(new SqlFieldsQuery("insert into a(col1, col2) values (2, ?1)")
+                .setArgs(VERY_SENS_STR_DATA));
         }
-        finally {
-            GridTestUtils.setFieldValue(S.class.getField("INCLUDE_SENSITIVE"), curSensVal);
+        catch (Throwable e) {
+            assertTrue(X.hasCause(e, CorruptedTreeException.class));
         }
+
+        assertFalse(logListener.check());
+
+        assertTrue(logSensListener.check());
     }
 }