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/03/28 13:31:31 UTC

[ignite] branch master updated: IGNITE-11536 Add information about possible long GC pause to checkpoint started message. - Fixes #6276.

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 35a93c7  IGNITE-11536 Add information about possible long GC pause to checkpoint started message. - Fixes #6276.
35a93c7 is described below

commit 35a93c75a5b5a8ecaf1eb4ab7280d7166029c638
Author: Sergey Antonov <an...@gmail.com>
AuthorDate: Thu Mar 28 16:26:45 2019 +0300

    IGNITE-11536 Add information about possible long GC pause to checkpoint started message. - Fixes #6276.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../apache/ignite/internal/GridKernalContext.java  |  7 ++
 .../ignite/internal/GridKernalContextImpl.java     | 18 +++--
 .../org/apache/ignite/internal/IgniteKernal.java   |  3 +-
 .../ignite/internal/LongJVMPauseDetector.java      | 64 ++++++++++++++++--
 .../GridCacheDatabaseSharedManager.java            | 78 +++++++++++++++++-----
 .../pagemem/CheckpointMetricsTracker.java          |  7 ++
 .../wal/reader/StandaloneGridKernalContext.java    |  6 ++
 .../junits/GridTestKernalContext.java              |  4 +-
 8 files changed, 159 insertions(+), 28 deletions(-)

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 744f858..ec9408e 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
@@ -465,6 +465,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public DataStructuresProcessor dataStructures();
 
     /**
+     * Gets long JVM pause detector.
+     *
+     * @return Long JVM pause detector.
+     */
+    public LongJVMPauseDetector longJvmPauseDetector();
+
+    /**
      * Checks whether this node is invalid due to a critical error or not.
      *
      * @return {@code True} if this node is invalid, {@code false} otherwise.
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 85e02f9..3e0f231 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
@@ -304,9 +304,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     @GridToStringExclude
     private ClusterProcessor cluster;
 
-    /**
-     *
-     */
+    /** */
     @GridToStringExclude
     private CompressionProcessor compressProc;
 
@@ -395,6 +393,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     private WorkersRegistry workersRegistry;
 
     /** */
+    @GridToStringExclude
+    private LongJVMPauseDetector pauseDetector;
+
+    /** */
     private Thread.UncaughtExceptionHandler hnd;
 
     /** */
@@ -471,6 +473,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      * @param plugins Plugin providers.
      * @param workerRegistry Worker registry.
      * @param hnd Default uncaught exception handler used by thread pools.
+     * @param pauseDetector Long JVM pause detector.
      */
     @SuppressWarnings("TypeMayBeWeakened")
     protected GridKernalContextImpl(
@@ -497,7 +500,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         List<PluginProvider> plugins,
         IgnitePredicate<String> clsFilter,
         WorkersRegistry workerRegistry,
-        Thread.UncaughtExceptionHandler hnd
+        Thread.UncaughtExceptionHandler hnd,
+        LongJVMPauseDetector pauseDetector
     ) {
         assert grid != null;
         assert cfg != null;
@@ -524,6 +528,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.customExecSvcs = customExecSvcs;
         this.workersRegistry = workerRegistry;
         this.hnd = hnd;
+        this.pauseDetector = pauseDetector;
 
         marshCtx = new MarshallerContextImpl(plugins, clsFilter);
 
@@ -971,6 +976,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public LongJVMPauseDetector longJvmPauseDetector() {
+        return pauseDetector;
+    }
+
+    /** {@inheritDoc} */
     @Override public void printMemoryStats() {
         X.println(">>> ");
         X.println(">>> Grid memory stats [igniteInstanceName=" + igniteInstanceName() + ']');
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 9462c50..3b8604f 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
@@ -929,7 +929,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 plugins,
                 MarshallerUtils.classNameFilter(this.getClass().getClassLoader()),
                 workerRegistry,
-                hnd
+                hnd,
+                longJVMPauseDetector
             );
 
             cfg.getMarshaller().setContext(ctx.marshallerContext());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/LongJVMPauseDetector.java b/modules/core/src/main/java/org/apache/ignite/internal/LongJVMPauseDetector.java
index d98b6d9..50346db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/LongJVMPauseDetector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/LongJVMPauseDetector.java
@@ -21,8 +21,11 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_JVM_PAUSE_DETECTOR_DISABLED;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_JVM_PAUSE_DETECTOR_LAST_EVENTS_COUNT;
@@ -40,16 +43,24 @@ import static org.apache.ignite.IgniteSystemProperties.getInteger;
  * configured in system or environment properties IGNITE_JVM_PAUSE_DETECTOR_PRECISION,
  * IGNITE_JVM_PAUSE_DETECTOR_THRESHOLD and IGNITE_JVM_PAUSE_DETECTOR_LAST_EVENTS_COUNT accordingly.
  */
-class LongJVMPauseDetector {
+public class LongJVMPauseDetector {
+    /** Ignite JVM pause detector threshold default value. */
+    public static final int DEFAULT_JVM_PAUSE_DETECTOR_THRESHOLD = 500;
+
     /** Precision. */
     private static final int PRECISION = getInteger(IGNITE_JVM_PAUSE_DETECTOR_PRECISION, 50);
 
     /** Threshold. */
-    private static final int THRESHOLD = getInteger(IGNITE_JVM_PAUSE_DETECTOR_THRESHOLD, 500);
+    private static final int THRESHOLD =
+        getInteger(IGNITE_JVM_PAUSE_DETECTOR_THRESHOLD, DEFAULT_JVM_PAUSE_DETECTOR_THRESHOLD);
 
     /** Event count. */
     private static final int EVT_CNT = getInteger(IGNITE_JVM_PAUSE_DETECTOR_LAST_EVENTS_COUNT, 20);
 
+    /** Disabled flag. */
+    private static final boolean DISABLED =
+        getBoolean(IGNITE_JVM_PAUSE_DETECTOR_DISABLED, false);
+
     /** Logger. */
     private final IgniteLogger log;
 
@@ -62,6 +73,9 @@ class LongJVMPauseDetector {
     /** Long pause total duration. */
     private long longPausesTotalDuration;
 
+    /** Last detector's wake up time. */
+    private long lastWakeUpTime;
+
     /** Long pauses timestamps. */
     @GridToStringInclude
     private final long[] longPausesTimestamps = new long[EVT_CNT];
@@ -81,7 +95,7 @@ class LongJVMPauseDetector {
      * Starts worker if not started yet.
      */
     public void start() {
-        if (getBoolean(IGNITE_JVM_PAUSE_DETECTOR_DISABLED, false)) {
+        if (DISABLED) {
             if (log.isDebugEnabled())
                 log.debug("JVM Pause Detector is disabled.");
 
@@ -89,9 +103,12 @@ class LongJVMPauseDetector {
         }
 
         final Thread worker = new Thread("jvm-pause-detector-worker") {
-            private long prev = System.currentTimeMillis();
 
             @Override public void run() {
+                synchronized (LongJVMPauseDetector.this) {
+                    lastWakeUpTime = System.currentTimeMillis();
+                }
+
                 if (log.isDebugEnabled())
                     log.debug(getName() + " has been started.");
 
@@ -100,9 +117,7 @@ class LongJVMPauseDetector {
                         Thread.sleep(PRECISION);
 
                         final long now = System.currentTimeMillis();
-                        final long pause = now - PRECISION - prev;
-
-                        prev = now;
+                        final long pause = now - PRECISION - lastWakeUpTime;
 
                         if (pause >= THRESHOLD) {
                             log.warning("Possible too long JVM pause: " + pause + " milliseconds.");
@@ -117,6 +132,13 @@ class LongJVMPauseDetector {
                                 longPausesTimestamps[next] = now;
 
                                 longPausesDurations[next] = pause;
+
+                                lastWakeUpTime = now;
+                            }
+                        }
+                        else {
+                            synchronized (LongJVMPauseDetector.this) {
+                                lastWakeUpTime = now;
                             }
                         }
                     }
@@ -156,6 +178,14 @@ class LongJVMPauseDetector {
     }
 
     /**
+     * @return {@code false} if {@link IgniteSystemProperties#IGNITE_JVM_PAUSE_DETECTOR_DISABLED} set to {@code true},
+     * and {@code true} otherwise.
+     */
+    public static boolean enabled() {
+        return !DISABLED;
+    }
+
+    /**
      * @return Long JVM pauses count.
      */
     synchronized long longPausesCount() {
@@ -170,6 +200,13 @@ class LongJVMPauseDetector {
     }
 
     /**
+     * @return Last checker's wake up time.
+     */
+    public synchronized long getLastWakeUpTime() {
+        return lastWakeUpTime;
+    }
+
+    /**
      * @return Last long JVM pause events.
      */
     synchronized Map<Long, Long> longPauseEvents() {
@@ -181,6 +218,19 @@ class LongJVMPauseDetector {
         return evts;
     }
 
+    /**
+     * @return Pair ({@code last long pause event time}, {@code pause time duration}) or {@code null}, if long pause
+     * wasn't occurred.
+     */
+    public synchronized @Nullable IgniteBiTuple<Long, Long> getLastLongPause() {
+        int lastPauseIdx = (int)((EVT_CNT + longPausesCnt - 1) % EVT_CNT);
+
+        if (longPausesTimestamps[lastPauseIdx] == 0)
+            return null;
+
+        return new IgniteBiTuple<>(longPausesTimestamps[lastPauseIdx], longPausesDurations[lastPauseIdx]);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(LongJVMPauseDetector.class, this);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 55fa785..30d23a9 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -86,9 +86,11 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.LongJVMPauseDetector;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
@@ -187,12 +189,17 @@ import org.jsr166.ConcurrentLinkedHashMap;
 
 import static java.nio.file.StandardOpenOption.READ;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_JVM_PAUSE_DETECTOR_THRESHOLD;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_RECOVERY_SEMAPHORE_PERMITS;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
 import static org.apache.ignite.failure.FailureType.SYSTEM_CRITICAL_OPERATION_TIMEOUT;
 import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION;
+import static org.apache.ignite.internal.IgnitionEx.grid;
+import static org.apache.ignite.internal.LongJVMPauseDetector.DEFAULT_JVM_PAUSE_DETECTOR_THRESHOLD;
 import static org.apache.ignite.internal.pagemem.PageIdUtils.partId;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CHECKPOINT_RECORD;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.METASTORE_DATA_RECORD;
@@ -215,18 +222,17 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     public static final String METASTORE_DATA_REGION_NAME = "metastoreMemPlc";
 
     /** Skip sync. */
-    private final boolean skipSync = IgniteSystemProperties.getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC);
+    private final boolean skipSync = getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC);
 
     /** */
-    private final int walRebalanceThreshold = IgniteSystemProperties.getInteger(
-        IGNITE_PDS_WAL_REBALANCE_THRESHOLD, 500_000);
+    private final int walRebalanceThreshold = getInteger(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, 500_000);
 
     /** Value of property for throttling policy override. */
     private final String throttlingPolicyOverride = IgniteSystemProperties.getString(
         IgniteSystemProperties.IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED);
 
     /** */
-    private final boolean skipCheckpointOnNodeStop = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP, false);
+    private final boolean skipCheckpointOnNodeStop = getBoolean(IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP, false);
 
     /**
      * Starting from this number of dirty pages in checkpoint, array will be sorted with
@@ -385,8 +391,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private volatile long checkpointReadLockTimeout;
 
     /** Flag allows to log additional information about partitions during recovery phases. */
-    private final boolean recoveryVerboseLogging = IgniteSystemProperties.getBoolean(
-            IgniteSystemProperties.IGNITE_RECOVERY_VERBOSE_LOGGING, true);
+    private final boolean recoveryVerboseLogging =
+        getBoolean(IgniteSystemProperties.IGNITE_RECOVERY_VERBOSE_LOGGING, true);
 
     /** Pointer to a memory recovery record that should be included into the next checkpoint record. */
     private volatile WALPointer memoryRecoveryRecordPtr;
@@ -2362,7 +2368,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             permits = permits0;
 
         // Property for override any calculation.
-        return IgniteSystemProperties.getInteger(IGNITE_RECOVERY_SEMAPHORE_PERMITS, permits);
+        return getInteger(IGNITE_RECOVERY_SEMAPHORE_PERMITS, permits);
     }
 
     /**
@@ -3374,6 +3380,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      */
     @SuppressWarnings("NakedNotify")
     public class Checkpointer extends GridWorker {
+        /** Checkpoint started log message format. */
+        private static final String CHECKPOINT_STARTED_LOG_FORMAT = "Checkpoint started [checkpointId=%s, startPtr=%s," +
+            " checkpointBeforeLockTime=%dms, checkpointLockWait=%dms, checkpointListenersExecuteTime=%dms, " +
+            "checkpointLockHoldTime=%dms, walCpRecordFsyncDuration=%dms, %s pages=%d, reason='%s']";
+
         /** Temporary write buffer. */
         private final ByteBuffer tmpWriteBuf;
 
@@ -3389,6 +3400,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /** */
         private long lastCpTs;
 
+        /** Pause detector. */
+        private final LongJVMPauseDetector pauseDetector;
+
+        /** Long JVM pause threshold. */
+        private final int longJvmPauseThreshold =
+            getInteger(IGNITE_JVM_PAUSE_DETECTOR_THRESHOLD, DEFAULT_JVM_PAUSE_DETECTOR_THRESHOLD);
+
         /**
          * @param gridName Grid name.
          * @param name Thread name.
@@ -3402,6 +3420,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             tmpWriteBuf = ByteBuffer.allocateDirect(pageSize());
 
             tmpWriteBuf.order(ByteOrder.nativeOrder());
+
+            pauseDetector = cctx.kernalContext().longJvmPauseDetector();
         }
 
         /**
@@ -4018,12 +4038,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 GridMultiCollectionWrapper<FullPageId> cpPages = splitAndSortCpPagesIfNeeded(
                     cpPagesTuple, persistenceCfg.getCheckpointThreads());
 
-                if (printCheckpointStats)
-                    if (log.isInfoEnabled())
-                        log.info(String.format("Checkpoint started [checkpointId=%s, startPtr=%s, " +
-                                "checkpointBeforeLockTime=%dms, checkpointLockWait=%dms, " +
-                                "checkpointListenersExecuteTime=%dms, checkpointLockHoldTime=%dms, " +
-                                "walCpRecordFsyncDuration=%dms, pages=%d, reason='%s']",
+                if (printCheckpointStats && log.isInfoEnabled()) {
+                    long possibleJvmPauseDur = possibleLongJvmPauseDuration(tracker);
+
+                    log.info(
+                        String.format(
+                            CHECKPOINT_STARTED_LOG_FORMAT,
                             cpRec.checkpointId(),
                             cp.checkpointMark(),
                             tracker.beforeLockDuration(),
@@ -4031,9 +4051,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             tracker.listenersExecuteDuration(),
                             tracker.lockHoldDuration(),
                             tracker.walCpRecordFsyncDuration(),
+                            possibleJvmPauseDur > 0 ? "possibleJvmPauseDuration=" + possibleJvmPauseDur + "ms," : "",
                             cpPages.size(),
-                            curr.reason)
-                        );
+                            curr.reason
+                        )
+                    );
+                }
 
                 return new Checkpoint(cp, cpPages, curr);
             }
@@ -4058,6 +4081,31 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         /**
+         * @param tracker Checkpoint metrics tracker.
+         * @return Duration of possible JVM pause, if it was detected, or {@code -1} otherwise.
+         */
+        private long possibleLongJvmPauseDuration(CheckpointMetricsTracker tracker) {
+            if (LongJVMPauseDetector.enabled()) {
+                if (tracker.lockWaitDuration() + tracker.lockHoldDuration() > longJvmPauseThreshold) {
+                    long now = System.currentTimeMillis();
+
+                    // We must get last wake up time before search possible pause in events map.
+                    long wakeUpTime = pauseDetector.getLastWakeUpTime();
+
+                    IgniteBiTuple<Long, Long> lastLongPause = pauseDetector.getLastLongPause();
+
+                    if (lastLongPause != null && tracker.checkpointStartTime() < lastLongPause.get1())
+                        return lastLongPause.get2();
+
+                    if (now - wakeUpTime > longJvmPauseThreshold)
+                        return now - wakeUpTime;
+                }
+            }
+
+            return -1L;
+        }
+
+        /**
          * Take read lock for internal use.
          */
         private void internalReadUnlock() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java
index d7291b5..d0ffefd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java
@@ -232,4 +232,11 @@ public class CheckpointMetricsTracker {
     public long walCpRecordFsyncDuration() {
         return walCpRecordFsyncEnd - walCpRecordFsyncStart;
     }
+
+    /**
+     * @return Checkpoint start time.
+     */
+    public long checkpointStartTime() {
+        return cpStart;
+    }
 }
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 0396c3e..3ec8a69 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
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridKernalGateway;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.LongJVMPauseDetector;
 import org.apache.ignite.internal.MarshallerContextImpl;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager;
@@ -694,4 +695,9 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     @Override public CompressionProcessor compress() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public LongJVMPauseDetector longJvmPauseDetector() {
+        return new LongJVMPauseDetector(log);
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 1f95b94..3594b79 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.GridKernalContextImpl;
 import org.apache.ignite.internal.GridKernalGatewayImpl;
 import org.apache.ignite.internal.GridLoggerProxy;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.LongJVMPauseDetector;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -81,7 +82,8 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 U.allPluginProviders(),
                 null,
                 null,
-                null
+                null,
+                new LongJVMPauseDetector(log)
         );
 
         GridTestUtils.setFieldValue(grid(), "cfg", config());