You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2018/10/23 15:01:05 UTC

[01/50] [abbrv] ignite git commit: IGNITE-9737 Added configuration parameters for system workers liveness checking

Repository: ignite
Updated Branches:
  refs/heads/ignite-gg-14206 1a7a9c717 -> 1e91d477f


IGNITE-9737 Added configuration parameters for system workers liveness checking

Signed-off-by: Andrey Gura <ag...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/8828c1d1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/8828c1d1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/8828c1d1

Branch: refs/heads/ignite-gg-14206
Commit: 8828c1d1910d209b9f520efcc8d256c6bc6b4251
Parents: 33b9611
Author: Andrey Kuznetsov <st...@gmail.com>
Authored: Wed Oct 17 16:53:53 2018 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Wed Oct 17 16:58:07 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  11 +
 .../configuration/DataStorageConfiguration.java |  27 ++
 .../configuration/IgniteConfiguration.java      |  29 ++
 .../apache/ignite/internal/IgniteKernal.java    |   8 +
 .../org/apache/ignite/internal/IgnitionEx.java  |   6 +-
 .../GridCacheDatabaseSharedManager.java         | 139 +++++++---
 .../IgniteCacheDatabaseSharedManager.java       |  14 +
 .../processors/failure/FailureProcessor.java    |  12 +-
 .../utils/PlatformConfigurationUtils.java       |  42 ++-
 .../ignite/internal/util/IgniteUtils.java       |  11 +
 .../worker/FailureHandlingMxBeanImpl.java       |  73 ++++++
 .../worker/WorkersControlMXBeanImpl.java        |  10 -
 .../ignite/internal/worker/WorkersRegistry.java |  44 +++-
 .../ignite/mxbean/FailureHandlingMxBean.java    |  47 ++++
 .../ignite/mxbean/WorkersControlMXBean.java     |   7 -
 .../FailureHandlingConfigurationTest.java       | 262 +++++++++++++++++++
 .../IgniteBasicWithPersistenceTestSuite.java    |   2 +
 .../IgniteConfigurationTest.cs                  |   3 +
 .../Configuration/DataStorageConfiguration.cs   |   7 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  14 +
 .../IgniteConfigurationSection.xsd              |  14 +
 21 files changed, 697 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 02ebb25..521222c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1008,6 +1008,17 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_REUSE_MEMORY_ON_DEACTIVATE = "IGNITE_REUSE_MEMORY_ON_DEACTIVATE";
 
     /**
+     * Maximum inactivity period for system worker in milliseconds. When this value is exceeded, worker is considered
+     * blocked with consequent critical failure handler invocation.
+     */
+    public static final String IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT = "IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT";
+
+    /**
+     * Timeout for checkpoint read lock acquisition in milliseconds.
+     */
+    public static final String IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT = "IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT";
+
+    /**
      * Timeout for waiting schema update if schema was not found for last accepted version.
      */
     public static final String IGNITE_WAIT_SCHEMA_UPDATE = "IGNITE_WAIT_SCHEMA_UPDATE";

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
index 556e3cd..7bca0f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
@@ -279,6 +279,9 @@ public class DataStorageConfiguration implements Serializable {
      */
     private int walCompactionLevel = DFLT_WAL_COMPACTION_LEVEL;
 
+    /** Timeout for checkpoint read lock acquisition. */
+    private Long checkpointReadLockTimeout;
+
     /**
      * Initial size of a data region reserved for system cache.
      *
@@ -983,6 +986,30 @@ public class DataStorageConfiguration implements Serializable {
         this.walCompactionLevel = walCompactionLevel;
     }
 
+    /**
+     * Returns timeout for checkpoint read lock acquisition.
+     *
+     * @see #setCheckpointReadLockTimeout(long)
+     * @return Returns timeout for checkpoint read lock acquisition in milliseconds.
+     */
+    public Long getCheckpointReadLockTimeout() {
+        return checkpointReadLockTimeout;
+    }
+
+    /**
+     * Sets timeout for checkpoint read lock acquisition.
+     * <p>
+     * When any thread cannot acquire checkpoint read lock in this time, then critical failure handler is being called.
+     *
+     * @param checkpointReadLockTimeout Timeout for checkpoint read lock acquisition in milliseconds.
+     * @return {@code this} for chaining.
+     */
+    public DataStorageConfiguration setCheckpointReadLockTimeout(long checkpointReadLockTimeout) {
+        this.checkpointReadLockTimeout = checkpointReadLockTimeout;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataStorageConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
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 1dbec7d..e7ccaf5 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
@@ -411,6 +411,9 @@ public class IgniteConfiguration {
     /** Failure detection timeout. */
     private Long failureDetectionTimeout = DFLT_FAILURE_DETECTION_TIMEOUT;
 
+    /** Timeout for blocked system workers detection. */
+    private Long sysWorkerBlockedTimeout;
+
     /** Failure detection timeout for client nodes. */
     private Long clientFailureDetectionTimeout = DFLT_CLIENT_FAILURE_DETECTION_TIMEOUT;
 
@@ -624,6 +627,7 @@ public class IgniteConfiguration {
         svcCfgs = cfg.getServiceConfiguration();
         svcPoolSize = cfg.getServiceThreadPoolSize();
         sysPoolSize = cfg.getSystemThreadPoolSize();
+        sysWorkerBlockedTimeout = cfg.getSystemWorkerBlockedTimeout();
         timeSrvPortBase = cfg.getTimeServerPortBase();
         timeSrvPortRange = cfg.getTimeServerPortRange();
         txCfg = cfg.getTransactionConfiguration();
@@ -1982,6 +1986,31 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Returns maximum inactivity period for system worker. When this value is exceeded, worker is considered blocked
+     * with consequent critical failure handler invocation.
+     *
+     * @see #setSystemWorkerBlockedTimeout(long)
+     * @return Maximum inactivity period for system worker in milliseconds.
+     */
+    public Long getSystemWorkerBlockedTimeout() {
+        return sysWorkerBlockedTimeout;
+    }
+
+    /**
+     * Sets maximum inactivity period for system worker. When this value is exceeded, worker is considered blocked
+     * with consequent critical failure handler invocation.
+     *
+     * @see #setFailureHandler(FailureHandler)
+     * @param sysWorkerBlockedTimeout Maximum inactivity period for system worker in milliseconds.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setSystemWorkerBlockedTimeout(long sysWorkerBlockedTimeout) {
+        this.sysWorkerBlockedTimeout = sysWorkerBlockedTimeout;
+
+        return this;
+    }
+
+    /**
      * Should return fully configured load balancing SPI implementation. If not provided,
      * {@link RoundRobinLoadBalancingSpi} will be used.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 f065863..7f4310f 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
@@ -188,6 +188,7 @@ 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.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.worker.FailureHandlingMxBeanImpl;
 import org.apache.ignite.internal.worker.WorkersControlMXBeanImpl;
 import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -201,6 +202,7 @@ import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.MarshallerUtils;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.mxbean.FailureHandlingMxBean;
 import org.apache.ignite.mxbean.ClusterMetricsMXBean;
 import org.apache.ignite.mxbean.DataStorageMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
@@ -4365,6 +4367,12 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 registerMBean("Kernal", workerCtrlMXBean.getClass().getSimpleName(),
                     workerCtrlMXBean, WorkersControlMXBean.class);
             }
+
+            FailureHandlingMxBean blockOpCtrlMXBean = new FailureHandlingMxBeanImpl(workersRegistry,
+                ctx.cache().context().database());
+
+            registerMBean("Kernal", blockOpCtrlMXBean.getClass().getSimpleName(), blockOpCtrlMXBean,
+                FailureHandlingMxBean.class);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
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 95001de..1e56cde 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
@@ -138,6 +138,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_LOCAL_HOST;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_SHUTDOWN_HOOK;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_RESTART_CODE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
@@ -1830,7 +1831,10 @@ public class IgnitionEx {
                                 new IgniteException(S.toString(GridWorker.class, deadWorker))));
                     }
                 },
-                cfg.getFailureDetectionTimeout(),
+                IgniteSystemProperties.getLong(IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT,
+                    cfg.getSystemWorkerBlockedTimeout() != null
+                    ? cfg.getSystemWorkerBlockedTimeout()
+                    : cfg.getFailureDetectionTimeout()),
                 log);
 
             stripedExecSvc = new StripedExecutor(

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 e74954f..ea3e314 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
@@ -79,7 +79,6 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.NodeStoppingException;
-import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.DirectMemoryRegion;
@@ -111,9 +110,9 @@ import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntry;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntryType;
@@ -160,12 +159,13 @@ import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.mxbean.DataStorageMetricsMXBean;
 import org.apache.ignite.thread.IgniteThread;
-import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 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_PDS_SKIP_CRC;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
@@ -352,6 +352,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /** File I/O factory for writing checkpoint markers. */
     private final FileIOFactory ioFactory;
+
+    /** Timeout for checkpoint read lock acquisition in milliseconds. */
+    private volatile long checkpointReadLockTimeout;
+
     /**
      * @param ctx Kernal context.
      */
@@ -377,6 +381,17 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         );
 
         ioFactory = persistenceCfg.getFileIOFactory();
+
+        Long cfgCheckpointReadLockTimeout = ctx.config().getDataStorageConfiguration() != null
+            ? ctx.config().getDataStorageConfiguration().getCheckpointReadLockTimeout()
+            : null;
+
+        checkpointReadLockTimeout = IgniteSystemProperties.getLong(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT,
+            cfgCheckpointReadLockTimeout != null
+                ? cfgCheckpointReadLockTimeout
+                : (ctx.workersRegistry() != null
+                    ? ctx.workersRegistry().getSystemWorkerBlockedTimeout()
+                    : ctx.config().getFailureDetectionTimeout()));
     }
 
     /** */
@@ -1486,53 +1501,64 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         if (checkpointLock.writeLock().isHeldByCurrentThread())
             return;
 
-        long timeout = cctx.gridConfig().getFailureDetectionTimeout();
+        long timeout = checkpointReadLockTimeout;
 
         long start = U.currentTimeMillis();
-        long passed;
 
         boolean interruped = false;
 
         try {
             for (; ; ) {
-                if ((passed = U.currentTimeMillis() - start) >= timeout)
-                    failCheckpointReadLock();
-
                 try {
-                    if (!checkpointLock.readLock().tryLock(timeout - passed, TimeUnit.MILLISECONDS))
+                    if (timeout > 0 && (U.currentTimeMillis() - start) >= timeout)
                         failCheckpointReadLock();
-                }
-                catch (InterruptedException e) {
-                    interruped = true;
 
-                    continue;
-                }
+                    try {
+                        if (timeout > 0) {
+                            if (!checkpointLock.readLock().tryLock(timeout - (U.currentTimeMillis() - start),
+                                TimeUnit.MILLISECONDS))
+                                failCheckpointReadLock();
+                        }
+                        else
+                            checkpointLock.readLock().lock();
+                    }
+                    catch (InterruptedException e) {
+                        interruped = true;
 
-                if (stopping) {
-                    checkpointLock.readLock().unlock();
+                        continue;
+                    }
 
-                    throw new IgniteException(new NodeStoppingException("Failed to perform cache update: node is stopping."));
-                }
+                    if (stopping) {
+                        checkpointLock.readLock().unlock();
 
-                if (checkpointLock.getReadHoldCount() > 1 || safeToUpdatePageMemories())
-                    break;
-                else {
-                    checkpointLock.readLock().unlock();
+                        throw new IgniteException(new NodeStoppingException("Failed to perform cache update: node is stopping."));
+                    }
 
-                    if (U.currentTimeMillis() - start >= timeout)
-                        failCheckpointReadLock();
+                    if (checkpointLock.getReadHoldCount() > 1 || safeToUpdatePageMemories())
+                        break;
+                    else {
+                        checkpointLock.readLock().unlock();
 
-                    try {
-                        checkpointer.wakeupForCheckpoint(0, "too many dirty pages").cpBeginFut
-                            .getUninterruptibly();
-                    }
-                    catch (IgniteFutureTimeoutCheckedException e) {
-                        failCheckpointReadLock();
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IgniteException("Failed to wait for checkpoint begin.", e);
+                        if (timeout > 0 && U.currentTimeMillis() - start >= timeout)
+                            failCheckpointReadLock();
+
+                        try {
+                            checkpointer.wakeupForCheckpoint(0, "too many dirty pages").cpBeginFut
+                                .getUninterruptibly();
+                        }
+                        catch (IgniteFutureTimeoutCheckedException e) {
+                            failCheckpointReadLock();
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException("Failed to wait for checkpoint begin.", e);
+                        }
                     }
                 }
+                catch (CheckpointReadLockTimeoutException e) {
+                    log.error(e.getMessage(), e);
+
+                    timeout = 0;
+                }
             }
         }
         finally {
@@ -1544,13 +1570,21 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             CHECKPOINT_LOCK_HOLD_COUNT.set(CHECKPOINT_LOCK_HOLD_COUNT.get() + 1);
     }
 
-    /** */
-    private void failCheckpointReadLock() throws IgniteException {
-        IgniteException e = new IgniteException("Checkpoint read lock acquisition has been timed out.");
+    /**
+     * Invokes critical failure processing. Always throws.
+     *
+     * @throws CheckpointReadLockTimeoutException If node was not invalidated as result of handling.
+     * @throws IgniteException If node was invalidated as result of handling.
+     */
+    private void failCheckpointReadLock() throws CheckpointReadLockTimeoutException, IgniteException {
+        String msg = "Checkpoint read lock acquisition has been timed out.";
 
-        cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e));
+        IgniteException e = new IgniteException(msg);
 
-        throw e;
+        if (cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)))
+            throw e;
+
+        throw new CheckpointReadLockTimeoutException(msg);
     }
 
     /** {@inheritDoc} */
@@ -2883,6 +2917,24 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
+     * Timeout for checkpoint read lock acquisition.
+     *
+     * @return Timeout for checkpoint read lock acquisition in milliseconds.
+     */
+    @Override public long checkpointReadLockTimeout() {
+        return checkpointReadLockTimeout;
+    }
+
+    /**
+     * Sets timeout for checkpoint read lock acquisition.
+     *
+     * @param val New timeout in milliseconds, non-positive value denotes infinite timeout.
+     */
+    @Override public void checkpointReadLockTimeout(long val) {
+        checkpointReadLockTimeout = val;
+    }
+
+    /**
      * Partition destroy queue.
      */
     private static class PartitionDestroyQueue {
@@ -4809,4 +4861,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             super(lastArchivedSegment, log);
         }
     }
+
+    /** Indicates checkpoint read lock acquisition failure which did not lead to node invalidation. */
+    private static class CheckpointReadLockTimeoutException extends IgniteCheckedException {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private CheckpointReadLockTimeoutException(String msg) {
+            super(msg);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index f35d15a..28ce085 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -763,6 +763,20 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @return {@code 0} for non-persistent storage.
+     */
+    public long checkpointReadLockTimeout() {
+        return 0;
+    }
+
+    /**
+     * No-op for non-persistent storage.
+     */
+    public void checkpointReadLockTimeout(long val) {
+        // No-op.
+    }
+
+    /**
      * No-op for non-persistent storage.
      */
     public void cleanupCheckpointDirectory() throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
index b48eff1..f31f0e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
@@ -102,9 +102,10 @@ public class FailureProcessor extends GridProcessorAdapter {
      * Processes failure accordingly to configured {@link FailureHandler}.
      *
      * @param failureCtx Failure context.
+     * @return {@code True} If this very call led to Ignite node invalidation.
      */
-    public void process(FailureContext failureCtx) {
-        process(failureCtx, hnd);
+    public boolean process(FailureContext failureCtx) {
+        return process(failureCtx, hnd);
     }
 
     /**
@@ -112,13 +113,14 @@ public class FailureProcessor extends GridProcessorAdapter {
      *
      * @param failureCtx Failure context.
      * @param hnd Failure handler.
+     * @return {@code True} If this very call led to Ignite node invalidation.
      */
-    public synchronized void process(FailureContext failureCtx, FailureHandler hnd) {
+    public synchronized boolean process(FailureContext failureCtx, FailureHandler hnd) {
         assert failureCtx != null;
         assert hnd != null;
 
         if (this.failureCtx != null) // Node already terminating, no reason to process more errors.
-            return;
+            return false;
 
         U.error(ignite.log(), "Critical system error detected. Will be handled accordingly to configured handler " +
             "[hnd=" + hnd + ", failureCtx=" + failureCtx + ']', failureCtx.error());
@@ -136,5 +138,7 @@ public class FailureProcessor extends GridProcessorAdapter {
 
             log.error("Ignite node is in invalid state due to a critical failure.");
         }
+
+        return invalidated;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 25b9cb8..915075a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -251,9 +251,8 @@ public class PlatformConfigurationUtils {
         if (keyCnt > 0) {
             CacheKeyConfiguration[] keys = new CacheKeyConfiguration[keyCnt];
 
-            for (int i = 0; i < keyCnt; i++) {
+            for (int i = 0; i < keyCnt; i++)
                 keys[i] = new CacheKeyConfiguration(in.readString(), in.readString());
-            }
 
             ccfg.setKeyConfiguration(keys);
         }
@@ -662,6 +661,8 @@ public class PlatformConfigurationUtils {
             cfg.setMvccVacuumFrequency(in.readLong());
         if (in.readBoolean())
             cfg.setMvccVacuumThreadCount(in.readInt());
+        if (in.readBoolean())
+            cfg.setSystemWorkerBlockedTimeout(in.readLong());
 
         int sqlSchemasCnt = in.readInt();
 
@@ -1239,6 +1240,12 @@ public class PlatformConfigurationUtils {
         w.writeLong(cfg.getMvccVacuumFrequency());
         w.writeBoolean(true);
         w.writeInt(cfg.getMvccVacuumThreadCount());
+        if (cfg.getSystemWorkerBlockedTimeout() != null) {
+            w.writeBoolean(true);
+            w.writeLong(cfg.getSystemWorkerBlockedTimeout());
+        } else {
+            w.writeBoolean(false);
+        }
 
         if (cfg.getSqlSchemas() == null)
             w.writeInt(-1);
@@ -1894,21 +1901,22 @@ public class PlatformConfigurationUtils {
                 .setConcurrencyLevel(in.readInt())
                 .setWalAutoArchiveAfterInactivity(in.readLong());
 
+        if (in.readBoolean())
+            res.setCheckpointReadLockTimeout(in.readLong());
+
         int cnt = in.readInt();
 
         if (cnt > 0) {
             DataRegionConfiguration[] regs = new DataRegionConfiguration[cnt];
 
-            for (int i = 0; i < cnt; i++) {
+            for (int i = 0; i < cnt; i++)
                 regs[i] = readDataRegionConfiguration(in);
-            }
 
             res.setDataRegionConfigurations(regs);
         }
 
-        if (in.readBoolean()) {
+        if (in.readBoolean())
             res.setDefaultDataRegionConfiguration(readDataRegionConfiguration(in));
-        }
 
         return res;
     }
@@ -2022,25 +2030,31 @@ public class PlatformConfigurationUtils {
             w.writeInt(cfg.getConcurrencyLevel());
             w.writeLong(cfg.getWalAutoArchiveAfterInactivity());
 
+            if (cfg.getCheckpointReadLockTimeout() != null) {
+                w.writeBoolean(true);
+                w.writeLong(cfg.getCheckpointReadLockTimeout());
+            }
+            else
+                w.writeBoolean(false);
+
             if (cfg.getDataRegionConfigurations() != null) {
                 w.writeInt(cfg.getDataRegionConfigurations().length);
 
-                for (DataRegionConfiguration d : cfg.getDataRegionConfigurations()) {
+                for (DataRegionConfiguration d : cfg.getDataRegionConfigurations())
                     writeDataRegionConfiguration(w, d);
-                }
-            } else {
-                w.writeInt(0);
             }
+            else
+                w.writeInt(0);
 
             if (cfg.getDefaultDataRegionConfiguration() != null) {
                 w.writeBoolean(true);
                 writeDataRegionConfiguration(w, cfg.getDefaultDataRegionConfiguration());
-            } else {
-                w.writeBoolean(false);
             }
-        } else {
-            w.writeBoolean(false);
+            else
+                w.writeBoolean(false);
         }
+        else
+            w.writeBoolean(false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 edb9871..d2f45c9 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
@@ -8681,6 +8681,17 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * When {@code long} value given is positive returns that value, otherwise returns provided default value.
+     *
+     * @param i Input value.
+     * @param dflt Default value.
+     * @return {@code i} if {@code i > 0} and {@code dflt} otherwise.
+     */
+    public static long ensurePositive(long i, long dflt) {
+        return i <= 0 ? dflt : i;
+    }
+
+    /**
      * Gets wrapper class for a primitive type.
      *
      * @param cls Class. If {@code null}, method is no-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/worker/FailureHandlingMxBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/FailureHandlingMxBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/FailureHandlingMxBeanImpl.java
new file mode 100644
index 0000000..61b9afe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/FailureHandlingMxBeanImpl.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.worker;
+
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.mxbean.FailureHandlingMxBean;
+import org.jetbrains.annotations.NotNull;
+
+/** {@inheritDoc} */
+public class FailureHandlingMxBeanImpl implements FailureHandlingMxBean {
+    /** System worker registry. */
+    private final WorkersRegistry workerRegistry;
+
+    /** Database manager. */
+    private final IgniteCacheDatabaseSharedManager dbMgr;
+
+    /**
+     * @param workersRegistry Workers registry.
+     * @param dbMgr Database manager.
+     */
+    public FailureHandlingMxBeanImpl(
+        @NotNull WorkersRegistry workersRegistry,
+        @NotNull IgniteCacheDatabaseSharedManager dbMgr
+    ) {
+        this.workerRegistry = workersRegistry;
+        this.dbMgr = dbMgr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean getLivenessCheckEnabled() {
+        return workerRegistry.livenessCheckEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setLivenessCheckEnabled(boolean val) {
+        workerRegistry.livenessCheckEnabled(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSystemWorkerBlockedTimeout() {
+        return workerRegistry.getSystemWorkerBlockedTimeout();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setSystemWorkerBlockedTimeout(long val) {
+        workerRegistry.setSystemWorkerBlockedTimeout(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCheckpointReadLockTimeout() {
+        return dbMgr.checkpointReadLockTimeout();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setCheckpointReadLockTimeout(long val) {
+        dbMgr.checkpointReadLockTimeout(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
index e6abe6e..1f082b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
@@ -67,16 +67,6 @@ public class WorkersControlMXBeanImpl implements WorkersControlMXBean {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean getHealthMonitoringEnabled() {
-        return workerRegistry.livenessCheckEnabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setHealthMonitoringEnabled(boolean val) {
-        workerRegistry.livenessCheckEnabled(val);
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean stopThreadByUniqueName(String name) {
         Thread[] threads = Thread.getAllStackTraces().keySet().stream()
             .filter(t -> Objects.equals(t.getName(), name))

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
index 848bb59..153b289 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
@@ -61,8 +61,10 @@ public class WorkersRegistry implements GridWorkerListener {
     /** */
     private final IgniteBiInClosure<GridWorker, FailureType> workerFailedHnd;
 
-    /** Worker heartbeat timeout in milliseconds, when exceeded, worker is considered as blocked. */
-    private final long heartbeatTimeout;
+    /**
+     * Maximum inactivity period for system worker in milliseconds, when exceeded, worker is considered as blocked.
+     */
+    private volatile long sysWorkerBlockedTimeout;
 
     /** Time in milliseconds between successive workers checks. */
     private final long checkInterval;
@@ -72,15 +74,17 @@ public class WorkersRegistry implements GridWorkerListener {
 
     /**
      * @param workerFailedHnd Closure to invoke on worker failure.
-     * @param heartbeatTimeout Maximum allowed worker heartbeat interval in milliseconds, should be positive.
+     * @param sysWorkerBlockedTimeout Maximum allowed worker heartbeat interval in milliseconds, non-positive value denotes
+     * infinite interval.
      */
     public WorkersRegistry(
         @NotNull IgniteBiInClosure<GridWorker, FailureType> workerFailedHnd,
-        long heartbeatTimeout,
-        IgniteLogger log) {
+        long sysWorkerBlockedTimeout,
+        IgniteLogger log
+    ) {
         this.workerFailedHnd = workerFailedHnd;
-        this.heartbeatTimeout = heartbeatTimeout;
-        this.checkInterval = Math.min(DFLT_CHECK_INTERVAL, heartbeatTimeout);
+        this.sysWorkerBlockedTimeout = U.ensurePositive(sysWorkerBlockedTimeout, Long.MAX_VALUE);
+        this.checkInterval = Math.min(DFLT_CHECK_INTERVAL, sysWorkerBlockedTimeout);
         this.log = log;
     }
 
@@ -127,15 +131,33 @@ public class WorkersRegistry implements GridWorkerListener {
     }
 
     /** */
-    boolean livenessCheckEnabled() {
+    public boolean livenessCheckEnabled() {
         return livenessCheckEnabled;
     }
 
     /** */
-    void livenessCheckEnabled(boolean val) {
+    public void livenessCheckEnabled(boolean val) {
         livenessCheckEnabled = val;
     }
 
+    /**
+     * Returns maximum inactivity period for system worker. When exceeded, worker is considered as blocked.
+     *
+     * @return Maximum inactivity period for system worker in milliseconds.
+     */
+    public long getSystemWorkerBlockedTimeout() {
+        return sysWorkerBlockedTimeout == Long.MAX_VALUE ? 0 : sysWorkerBlockedTimeout;
+    }
+
+    /**
+     * Sets maximum inactivity period for system worker. When exceeded, worker is considered as blocked.
+     *
+     * @param val Maximum inactivity period for system worker in milliseconds.
+     */
+    public void setSystemWorkerBlockedTimeout(long val) {
+        sysWorkerBlockedTimeout = U.ensurePositive(val, Long.MAX_VALUE);
+    }
+
     /** {@inheritDoc} */
     @Override public void onStarted(GridWorker w) {
         register(w);
@@ -164,7 +186,7 @@ public class WorkersRegistry implements GridWorkerListener {
         try {
             lastCheckTs = U.currentTimeMillis();
 
-            long workersToCheck = Math.max(registeredWorkers.size() * checkInterval / heartbeatTimeout, 1);
+            long workersToCheck = Math.max(registeredWorkers.size() * checkInterval / sysWorkerBlockedTimeout, 1);
 
             int workersChecked = 0;
 
@@ -198,7 +220,7 @@ public class WorkersRegistry implements GridWorkerListener {
 
                     long heartbeatDelay = U.currentTimeMillis() - worker.heartbeatTs();
 
-                    if (heartbeatDelay > heartbeatTimeout) {
+                    if (heartbeatDelay > sysWorkerBlockedTimeout) {
                         GridWorker worker0 = registeredWorkers.get(worker.runner().getName());
 
                         if (worker0 != null && worker0 == worker) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/mxbean/FailureHandlingMxBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/FailureHandlingMxBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/FailureHandlingMxBean.java
new file mode 100644
index 0000000..199d752
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/FailureHandlingMxBean.java
@@ -0,0 +1,47 @@
+/*
+ * 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.mxbean;
+
+/**
+ * MBean that controls critical failure handling.
+ */
+@MXBeanDescription("MBean that controls critical failure handling.")
+public interface FailureHandlingMxBean {
+    /** */
+    @MXBeanDescription("Enable/disable critical workers liveness checking.")
+    public boolean getLivenessCheckEnabled();
+
+    /** */
+    public void setLivenessCheckEnabled(boolean val);
+
+    /** */
+    @MXBeanDescription("Maximum inactivity period for system worker. Critical failure handler fires if exceeded. " +
+        "Nonpositive value denotes infinite timeout.")
+    public long getSystemWorkerBlockedTimeout();
+
+    /** */
+    public void setSystemWorkerBlockedTimeout(long val);
+
+    /** */
+    @MXBeanDescription("Timeout for checkpoint read lock acquisition. Critical failure handler fires if exceeded. " +
+        "Nonpositive value denotes infinite timeout.")
+    public long getCheckpointReadLockTimeout();
+
+    /** */
+    public void setCheckpointReadLockTimeout(long val);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
index 18b0084..b999ab7 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
@@ -47,13 +47,6 @@ public interface WorkersControlMXBean {
     )
     public boolean terminateWorker(String name);
 
-    /** */
-    @MXBeanDescription("Whether workers check each other's health.")
-    public boolean getHealthMonitoringEnabled();
-
-    /** */
-    public void setHealthMonitoringEnabled(boolean val);
-
     /**
      * Stops thread by {@code name}, if exists and unique.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlingConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlingConfigurationTest.java
new file mode 100644
index 0000000..4e5b753
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlingConfigurationTest.java
@@ -0,0 +1,262 @@
+/*
+ * 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.failure;
+
+import java.lang.management.ManagementFactory;
+import java.util.concurrent.CountDownLatch;
+import javax.management.MBeanServer;
+import javax.management.MBeanServerInvocationHandler;
+import javax.management.ObjectName;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.worker.FailureHandlingMxBeanImpl;
+import org.apache.ignite.internal.worker.WorkersRegistry;
+import org.apache.ignite.mxbean.FailureHandlingMxBean;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT;
+
+/**
+ * Tests configuration parameters related to failure handling.
+ */
+public class FailureHandlingConfigurationTest extends GridCommonAbstractTest {
+    /** */
+    private Long checkpointReadLockTimeout;
+
+    /** */
+    private Long sysWorkerBlockedTimeout;
+
+    /** */
+    private CountDownLatch failureLatch;
+
+    /** */
+    private class TestFailureHandler extends AbstractFailureHandler {
+        /** */
+        TestFailureHandler() {
+            failureLatch = new CountDownLatch(1);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected boolean handle(Ignite ignite, FailureContext failureCtx) {
+            failureLatch.countDown();
+
+            return false;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setFailureHandler(new TestFailureHandler());
+
+        DataRegionConfiguration drCfg = new DataRegionConfiguration();
+        drCfg.setPersistenceEnabled(true);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setDefaultDataRegionConfiguration(drCfg);
+
+        if (checkpointReadLockTimeout != null)
+            dsCfg.setCheckpointReadLockTimeout(checkpointReadLockTimeout);
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        if (sysWorkerBlockedTimeout != null)
+            cfg.setSystemWorkerBlockedTimeout(sysWorkerBlockedTimeout);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        cleanPersistenceDir();
+
+        sysWorkerBlockedTimeout = null;
+        checkpointReadLockTimeout = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCfgParamsPropagation() throws Exception {
+        sysWorkerBlockedTimeout = 30_000L;
+        checkpointReadLockTimeout = 20_000L;
+
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        WorkersRegistry reg = ignite.context().workersRegistry();
+
+        IgniteCacheDatabaseSharedManager dbMgr = ignite.context().cache().context().database();
+
+        FailureHandlingMxBean mBean = getMBean();
+
+        assertEquals(sysWorkerBlockedTimeout.longValue(), reg.getSystemWorkerBlockedTimeout());
+        assertEquals(checkpointReadLockTimeout.longValue(), dbMgr.checkpointReadLockTimeout());
+
+        assertEquals(sysWorkerBlockedTimeout.longValue(), mBean.getSystemWorkerBlockedTimeout());
+        assertEquals(checkpointReadLockTimeout.longValue(), mBean.getCheckpointReadLockTimeout());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartialCfgParamsPropagation() throws Exception {
+        sysWorkerBlockedTimeout = 30_000L;
+        checkpointReadLockTimeout = null;
+
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        WorkersRegistry reg = ignite.context().workersRegistry();
+
+        IgniteCacheDatabaseSharedManager dbMgr = ignite.context().cache().context().database();
+
+        FailureHandlingMxBean mBean = getMBean();
+
+        assertEquals(sysWorkerBlockedTimeout.longValue(), reg.getSystemWorkerBlockedTimeout());
+        assertEquals(sysWorkerBlockedTimeout.longValue(), dbMgr.checkpointReadLockTimeout());
+
+        assertEquals(sysWorkerBlockedTimeout.longValue(), mBean.getSystemWorkerBlockedTimeout());
+        assertEquals(sysWorkerBlockedTimeout.longValue(), mBean.getCheckpointReadLockTimeout());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeParamValues() throws Exception {
+        sysWorkerBlockedTimeout = -1L;
+        checkpointReadLockTimeout = -85L;
+
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        WorkersRegistry reg = ignite.context().workersRegistry();
+
+        IgniteCacheDatabaseSharedManager dbMgr = ignite.context().cache().context().database();
+
+        FailureHandlingMxBean mBean = getMBean();
+
+        assertEquals(0L, reg.getSystemWorkerBlockedTimeout());
+        assertEquals(-85L, dbMgr.checkpointReadLockTimeout());
+
+        assertEquals(0L, mBean.getSystemWorkerBlockedTimeout());
+        assertEquals(-85L, mBean.getCheckpointReadLockTimeout());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverridingBySysProps() throws Exception {
+        String prevWorkerProp = System.getProperty(IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT);
+        String prevCheckpointProp = System.getProperty(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT);
+
+        long workerPropVal = 80_000;
+        long checkpointPropVal = 90_000;
+
+        System.setProperty(IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT, String.valueOf(workerPropVal));
+        System.setProperty(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT, String.valueOf(checkpointPropVal));
+
+        try {
+            sysWorkerBlockedTimeout = 1L;
+            checkpointReadLockTimeout = 2L;
+
+            IgniteEx ignite = startGrid(0);
+
+            ignite.cluster().active(true);
+
+            WorkersRegistry reg = ignite.context().workersRegistry();
+
+            IgniteCacheDatabaseSharedManager dbMgr = ignite.context().cache().context().database();
+
+            FailureHandlingMxBean mBean = getMBean();
+
+            assertEquals(sysWorkerBlockedTimeout, ignite.configuration().getSystemWorkerBlockedTimeout());
+            assertEquals(checkpointReadLockTimeout,
+                ignite.configuration().getDataStorageConfiguration().getCheckpointReadLockTimeout());
+
+            assertEquals(workerPropVal, reg.getSystemWorkerBlockedTimeout());
+            assertEquals(checkpointPropVal, dbMgr.checkpointReadLockTimeout());
+
+            assertEquals(workerPropVal, mBean.getSystemWorkerBlockedTimeout());
+            assertEquals(checkpointPropVal, mBean.getCheckpointReadLockTimeout());
+        }
+        finally {
+            if (prevWorkerProp != null)
+                System.setProperty(IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT, prevWorkerProp);
+            else
+                System.clearProperty(IGNITE_SYSTEM_WORKER_BLOCKED_TIMEOUT);
+
+            if (prevCheckpointProp != null)
+                System.setProperty(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT, prevCheckpointProp);
+            else
+                System.clearProperty(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMBeanParamsChanging() throws Exception {
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        FailureHandlingMxBean mBean = getMBean();
+
+        mBean.setSystemWorkerBlockedTimeout(80_000L);
+        assertEquals(80_000L, ignite.context().workersRegistry().getSystemWorkerBlockedTimeout());
+
+        mBean.setCheckpointReadLockTimeout(90_000L);
+        assertEquals(90_000L, ignite.context().cache().context().database().checkpointReadLockTimeout());
+
+        assertTrue(mBean.getLivenessCheckEnabled());
+        mBean.setLivenessCheckEnabled(false);
+        assertFalse(ignite.context().workersRegistry().livenessCheckEnabled());
+        ignite.context().workersRegistry().livenessCheckEnabled(true);
+        assertTrue(mBean.getLivenessCheckEnabled());
+    }
+
+    /** */
+    private FailureHandlingMxBean getMBean() throws Exception {
+        ObjectName name = U.makeMBeanName(getTestIgniteInstanceName(0), "Kernal",
+            FailureHandlingMxBeanImpl.class.getSimpleName());
+
+        MBeanServer srv = ManagementFactory.getPlatformMBeanServer();
+
+        assertTrue(srv.isRegistered(name));
+
+        return MBeanServerInvocationHandler.newProxyInstance(srv, name, FailureHandlingMxBean.class, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java
index e7876f8..68085db 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import java.util.Set;
 import junit.framework.TestSuite;
+import org.apache.ignite.failure.FailureHandlingConfigurationTest;
 import org.apache.ignite.failure.IoomFailureHandlerTest;
 import org.apache.ignite.failure.SystemWorkersTerminationTest;
 import org.apache.ignite.internal.ClusterBaselineNodesMetricsSelfTest;
@@ -63,6 +64,7 @@ public class IgniteBasicWithPersistenceTestSuite extends TestSuite {
         suite.addTestSuite(ServiceDeploymentOutsideBaselineTest.class);
         suite.addTestSuite(GridMarshallerMappingConsistencyTest.class);
         suite.addTestSuite(SystemWorkersTerminationTest.class);
+        suite.addTestSuite(FailureHandlingConfigurationTest.class);
 
         suite.addTestSuite(GridCommandHandlerTest.class);
         suite.addTestSuite(GridInternalTaskUnusedWalSegmentsTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 354a511..c26e5a3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -204,6 +204,7 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(com.UnacknowledgedMessagesBufferSize, resCom.UnacknowledgedMessagesBufferSize);
 
                 Assert.AreEqual(cfg.FailureDetectionTimeout, resCfg.FailureDetectionTimeout);
+                Assert.AreEqual(cfg.SystemWorkerBlockedTimeout, resCfg.SystemWorkerBlockedTimeout);
                 Assert.AreEqual(cfg.ClientFailureDetectionTimeout, resCfg.ClientFailureDetectionTimeout);
                 Assert.AreEqual(cfg.LongQueryWarningTimeout, resCfg.LongQueryWarningTimeout);
 
@@ -752,6 +753,7 @@ namespace Apache.Ignite.Core.Tests
                     UnacknowledgedMessagesBufferSize = 3450
                 },
                 FailureDetectionTimeout = TimeSpan.FromSeconds(3.5),
+                SystemWorkerBlockedTimeout = TimeSpan.FromSeconds(8.5),
                 ClientFailureDetectionTimeout = TimeSpan.FromMinutes(12.3),
                 LongQueryWarningTimeout = TimeSpan.FromMinutes(1.23),
                 IsActiveOnStart = true,
@@ -824,6 +826,7 @@ namespace Apache.Ignite.Core.Tests
                     ConcurrencyLevel = 1,
                     PageSize = 8 * 1024,
                     WalAutoArchiveAfterInactivity = TimeSpan.FromMinutes(5),
+                    CheckpointReadLockTimeout = TimeSpan.FromSeconds(9.5),
                     DefaultDataRegionConfiguration = new DataRegionConfiguration
                     {
                         Name = "reg1",

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
index 0a010b4..8771c77 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
@@ -234,6 +234,7 @@ namespace Apache.Ignite.Core.Configuration
             PageSize = reader.ReadInt();
             ConcurrencyLevel = reader.ReadInt();
             WalAutoArchiveAfterInactivity = reader.ReadLongAsTimespan();
+            CheckpointReadLockTimeout = reader.ReadTimeSpanNullable();
 
             var count = reader.ReadInt();
 
@@ -286,6 +287,7 @@ namespace Apache.Ignite.Core.Configuration
             writer.WriteInt(PageSize);
             writer.WriteInt(ConcurrencyLevel);
             writer.WriteTimeSpanAsLong(WalAutoArchiveAfterInactivity);
+            writer.WriteTimeSpanAsLongNullable(CheckpointReadLockTimeout);
 
             if (DataRegionConfigurations != null)
             {
@@ -489,6 +491,11 @@ namespace Apache.Ignite.Core.Configuration
         public TimeSpan WalAutoArchiveAfterInactivity { get; set; }
 
         /// <summary>
+        /// Gets or sets the timeout for checkpoint read lock acquisition.
+        /// </summary>
+        public TimeSpan? CheckpointReadLockTimeout { get; set; }
+
+        /// <summary>
         /// Gets or sets the data region configurations.
         /// </summary>
         [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index fc6afb6..63bf794 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -166,6 +166,9 @@ namespace Apache.Ignite.Core
         private TimeSpan? _clientFailureDetectionTimeout;
 
         /** */
+        private TimeSpan? _sysWorkerBlockedTimeout;
+
+        /** */
         private int? _publicThreadPoolSize;
 
         /** */
@@ -329,6 +332,7 @@ namespace Apache.Ignite.Core
             writer.WriteBooleanNullable(_authenticationEnabled);
             writer.WriteLongNullable(_mvccVacuumFreq);
             writer.WriteIntNullable(_mvccVacuumThreadCnt);
+            writer.WriteTimeSpanAsLongNullable(_sysWorkerBlockedTimeout);
 
             if (SqlSchemas == null)
                 writer.WriteInt(-1);
@@ -717,6 +721,7 @@ namespace Apache.Ignite.Core
             _authenticationEnabled = r.ReadBooleanNullable();
             _mvccVacuumFreq = r.ReadLongNullable();
             _mvccVacuumThreadCnt = r.ReadIntNullable();
+            _sysWorkerBlockedTimeout = r.ReadTimeSpanNullable();
 
             int sqlSchemasCnt = r.ReadInt();
 
@@ -1375,6 +1380,15 @@ namespace Apache.Ignite.Core
         }
 
         /// <summary>
+        /// Gets or sets the timeout for blocked system workers detection.
+        /// </summary>
+        public TimeSpan? SystemWorkerBlockedTimeout
+        {
+            get { return _sysWorkerBlockedTimeout; }
+            set { _sysWorkerBlockedTimeout = value; }
+        }
+
+        /// <summary>
         /// Gets or sets the failure detection timeout used by <see cref="TcpDiscoverySpi"/>
         /// and <see cref="TcpCommunicationSpi"/> for client nodes.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8828c1d1/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 0a55095..5f4a439 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1922,6 +1922,13 @@
                                 <xs:documentation>Inactivity time after which to run WAL segment auto archiving.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
+                        <xs:attribute name="checkpointReadLockTimeout" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>
+                                    Timeout for checkpoint read lock acquisition.
+                                </xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
                     </xs:complexType>
                 </xs:element>
                 <xs:element name="sslContextFactory" minOccurs="0">
@@ -2228,6 +2235,13 @@
                     </xs:documentation>
                 </xs:annotation>
             </xs:attribute>
+            <xs:attribute name="systemWorkerBlockedTimeout" type="xs:string">
+                <xs:annotation>
+                    <xs:documentation>
+                        Timeout for blocked system workers detection.
+                    </xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
             <xs:attribute name="clientFailureDetectionTimeout" type="xs:string">
                 <xs:annotation>
                     <xs:documentation>


[36/50] [abbrv] ignite git commit: IGNITE-9944: MVCC: Fixed "first request" handling for DHT transactions. This closes #5040.

Posted by sb...@apache.org.
IGNITE-9944: MVCC: Fixed "first request" handling for DHT transactions. This closes #5040.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7504880a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7504880a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7504880a

Branch: refs/heads/ignite-gg-14206
Commit: 7504880a443737bf03bb03e455dbea7d8757ae27
Parents: 4bc08da
Author: ipavlukhin <vo...@gmail.com>
Authored: Mon Oct 22 22:15:20 2018 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Mon Oct 22 22:15:20 2018 +0300

----------------------------------------------------------------------
 .../dht/GridDhtTxAbstractEnlistFuture.java      |  8 +-
 .../CacheMvccRemoteTxOnNearNodeStartTest.java   | 90 ++++++++++++++++++++
 .../testsuites/IgniteCacheMvccTestSuite.java    |  3 +
 3 files changed, 100 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7504880a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
index e2c8237..68669b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
@@ -193,6 +193,9 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
     /** Moving partitions. */
     private Map<Integer, Boolean> movingParts;
 
+    /** Map for tracking nodes to which first request was already sent in order to send smaller subsequent requests. */
+    private final Set<ClusterNode> firstReqSent = new HashSet<>();
+
     /**
      * @param nearNodeId Near node ID.
      * @param nearLockVer Near lock version.
@@ -823,9 +826,12 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
 
         GridDhtTxQueryEnlistRequest req;
 
-        if (newRemoteTx(node)) {
+        if (newRemoteTx(node))
             addNewRemoteTxNode(node);
 
+        if (!firstReqSent.contains(node)) {
+            firstReqSent.add(node);
+
             // If this is a first request to this node, send full info.
             req = new GridDhtTxQueryFirstEnlistRequest(cctx.cacheId(),
                 futId,

http://git-wip-us.apache.org/repos/asf/ignite/blob/7504880a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccRemoteTxOnNearNodeStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccRemoteTxOnNearNodeStartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccRemoteTxOnNearNodeStartTest.java
new file mode 100644
index 0000000..ee23e38
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccRemoteTxOnNearNodeStartTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.cache.mvcc;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/** */
+public class CacheMvccRemoteTxOnNearNodeStartTest extends CacheMvccAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /**
+     * Ensures that remote transaction on near node is started
+     * when first request is sent to OWNING partition and second to MOVING partition.
+     * @throws Exception if failed.
+     */
+    public void testRemoteTxOnNearNodeIsStartedIfPartitionIsMoving() throws Exception {
+        startGridsMultiThreaded(3);
+
+        IgniteCache<Object, Object> cache = grid(0).getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT)
+            .setCacheMode(cacheMode())
+            .setBackups(1)
+        );
+
+        ArrayList<Integer> keys = new ArrayList<>();
+
+        Affinity<Object> aff = grid(0).affinity(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(grid(1).localNode(), i) && aff.isBackup(grid(0).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(grid(1).localNode(), i) && aff.isBackup(grid(2).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        assert keys.size() == 2;
+
+        stopGrid(2);
+
+        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.putAll(ImmutableMap.of(
+                keys.get(0), 0,
+                keys.get(1), 1)
+            );
+
+            tx.commit();
+        }
+
+        // assert transaction was committed without errors
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7504880a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
index 8585ebe..0146344 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccOperationChecks
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedCoordinatorFailoverTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccProcessorLazyStartTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccProcessorTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccRemoteTxOnNearNodeStartTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedCoordinatorFailoverTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithConcurrentTransactionTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentTransactionTest;
@@ -52,6 +53,8 @@ public class IgniteCacheMvccTestSuite extends TestSuite {
         suite.addTestSuite(DataStreamProcessorMvccSelfTest.class);
         suite.addTestSuite(CacheMvccOperationChecksTest.class);
 
+        suite.addTestSuite(CacheMvccRemoteTxOnNearNodeStartTest.class);
+
         // Concurrent ops tests.
         suite.addTestSuite(CacheMvccIteratorWithConcurrentTransactionTest.class);
         suite.addTestSuite(CacheMvccLocalEntriesWithConcurrentTransactionTest.class);


[27/50] [abbrv] ignite git commit: IGNITE-9802 Fixed flaky TcpCommunicationSpiFaultyClientTest.testNotAcceptedConnection - Fixes #4931.

Posted by sb...@apache.org.
IGNITE-9802 Fixed flaky TcpCommunicationSpiFaultyClientTest.testNotAcceptedConnection - Fixes #4931.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3a4167a6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3a4167a6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3a4167a6

Branch: refs/heads/ignite-gg-14206
Commit: 3a4167a67e4fc318e2a7ef19500ca6da5910eaac
Parents: 5fa5c54
Author: NSAmelchev <ns...@gmail.com>
Authored: Mon Oct 22 14:45:38 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 14:45:38 2018 +0300

----------------------------------------------------------------------
 .../TcpCommunicationSpiFaultyClientTest.java    | 37 +++++++++++++++-----
 1 file changed, 29 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3a4167a6/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java
index 00b1d90..505d7a5 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java
@@ -23,6 +23,8 @@ import java.net.ServerSocket;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
@@ -38,6 +40,7 @@ import org.apache.ignite.internal.util.nio.GridCommunicationClient;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.spi.communication.CommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -79,7 +82,7 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest
         spi.setIdleConnectionTimeout(100);
         spi.setSharedMemoryPort(-1);
 
-        TcpDiscoverySpi discoSpi = (TcpDiscoverySpi) cfg.getDiscoverySpi();
+        TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
         discoSpi.setIpFinder(IP_FINDER);
         discoSpi.setClientReconnectDisabled(true);
@@ -94,11 +97,11 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL,"true");
+        System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL, "true");
     }
 
     /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
+    @Override protected void afterTestsStopped() {
         System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL);
     }
 
@@ -150,12 +153,30 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest
             startGrid(2);
             startGrid(3);
 
-            U.sleep(1000); // Wait for write timeout and closing idle connections.
+            // Need to wait for PME to avoid opening new connections during closing idle connections.
+            awaitPartitionMapExchange();
+
+            CommunicationSpi commSpi = grid(0).configuration().getCommunicationSpi();
+
+            ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(commSpi, "clients");
+
+            // Wait for write timeout and closing idle connections.
+            assertTrue("Failed to wait for closing idle connections.",
+                GridTestUtils.waitForCondition(() -> {
+                    for (GridCommunicationClient[] clients0 : clients.values()) {
+                        for (GridCommunicationClient client : clients0) {
+                            if (client != null)
+                                return false;
+                        }
+                    }
+
+                    return true;
+                }, 1000));
 
             final CountDownLatch latch = new CountDownLatch(1);
 
             grid(0).events().localListen(new IgnitePredicate<Event>() {
-                @Override public boolean apply(Event event) {
+                @Override public boolean apply(Event evt) {
                     latch.countDown();
 
                     return true;
@@ -171,7 +192,7 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest
                     }
                 });
             }
-            catch (IgniteException e) {
+            catch (IgniteException ignored) {
                 // No-op.
             }
 
@@ -218,7 +239,7 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest
          * Default constructor.
          */
         FakeServer() throws IOException {
-            this.srv = new ServerSocket(47200, 50, InetAddress.getByName("127.0.0.1"));
+            srv = new ServerSocket(47200, 50, InetAddress.getByName("127.0.0.1"));
         }
 
         /**
@@ -235,7 +256,7 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest
                     try {
                         U.sleep(10);
                     }
-                    catch (IgniteInterruptedCheckedException e) {
+                    catch (IgniteInterruptedCheckedException ignored) {
                         // No-op.
                     }
                 }


[05/50] [abbrv] ignite git commit: IGNITE-9272: j.u.zip.CRC32algo instead of PureJavaCrc32 - Fixes #4619.

Posted by sb...@apache.org.
IGNITE-9272: j.u.zip.CRC32algo instead of PureJavaCrc32 - Fixes #4619.

Signed-off-by: Nikolay Izhikov <ni...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/091ace90
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/091ace90
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/091ace90

Branch: refs/heads/ignite-gg-14206
Commit: 091ace90e56bac1f7bc76886a5dccff796cbe34e
Parents: 2bdc898
Author: Evgeny Stanilovskiy <es...@gridgain.com>
Authored: Wed Oct 17 17:56:08 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Wed Oct 17 17:56:08 2018 +0300

----------------------------------------------------------------------
 .../benchmarks/jmh/algo/BenchmarkCRC.java       |  95 +++++++++++++++++
 .../GridCacheDatabaseSharedManager.java         |   4 +-
 .../cache/persistence/file/EncryptedFileIO.java |   7 +-
 .../cache/persistence/file/FilePageStore.java   |   7 +-
 .../wal/FileWriteAheadLogManager.java           |   4 +-
 .../wal/FsyncModeFileWriteAheadLogManager.java  |   4 +-
 .../cache/persistence/wal/crc/FastCrc.java      | 101 +++++++++++++++++++
 .../persistence/wal/crc/PureJavaCrc32.java      |   2 +
 .../cache/persistence/wal/io/FileInput.java     |  10 +-
 .../persistence/wal/io/SimpleFileInput.java     |   3 +-
 .../wal/serializer/RecordV1Serializer.java      |   5 +-
 .../db/wal/crc/IgniteDataIntegrityTests.java    |   6 +-
 .../wal/crc/IgnitePureJavaCrcCompatibility.java |  55 ++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   2 +
 14 files changed, 284 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/algo/BenchmarkCRC.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/algo/BenchmarkCRC.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/algo/BenchmarkCRC.java
new file mode 100644
index 0000000..5c922fe
--- /dev/null
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/algo/BenchmarkCRC.java
@@ -0,0 +1,95 @@
+/*
+ * 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.benchmarks.jmh.algo;
+
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.openjdk.jmh.annotations.Mode.AverageTime;
+import static org.openjdk.jmh.annotations.Scope.Thread;
+
+/**
+ *
+ */
+@State(Thread)
+@OutputTimeUnit(NANOSECONDS)
+@BenchmarkMode(AverageTime)
+@Fork(value = 1, jvmArgsAppend = {"-XX:+UnlockDiagnosticVMOptions"})
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+public class BenchmarkCRC {
+    /** */
+    static final int SIZE = 1024;
+
+    /** */
+    static final int BUF_LEN = 4096;
+
+    /** */
+    @State(Thread)
+    public static class Context {
+        /** */
+        final int[] results = new int[SIZE];
+
+        /** */
+        final ByteBuffer bb = ByteBuffer.allocate(BUF_LEN);
+
+        /** */
+        @Setup
+        public void setup() {
+            new Random().ints(BUF_LEN, Byte.MIN_VALUE, Byte.MAX_VALUE).forEach(k -> bb.put((byte) k));
+        }
+    }
+
+    /** */
+    @Benchmark
+    public int[] pureJavaCrc32(Context context) {
+        for (int i = 0; i < SIZE; i++) {
+            context.bb.rewind();
+
+            context.results[i] = PureJavaCrc32.calcCrc32(context.bb, BUF_LEN);
+        }
+
+        return context.results;
+    }
+
+    /** */
+    @Benchmark
+    public int[] crc32(Context context) {
+        for (int i = 0; i < SIZE; i++) {
+            context.bb.rewind();
+
+            context.results[i] = FastCrc.calcCrc(context.bb, BUF_LEN);
+        }
+
+        return context.results;
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 ea3e314..33f593d 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
@@ -132,8 +132,8 @@ import org.apache.ignite.internal.processors.cache.persistence.snapshot.Snapshot
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
 import org.apache.ignite.internal.util.GridUnsafe;
@@ -4142,7 +4142,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     }
 
                     if (!skipCrc) {
-                        PageIO.setCrc(writeAddr, PureJavaCrc32.calcCrc32(tmpWriteBuf, pageSize()));
+                        PageIO.setCrc(writeAddr, FastCrc.calcCrc(tmpWriteBuf, pageSize()));
 
                         tmpWriteBuf.rewind();
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
index 008a728..86d9bbc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
@@ -21,10 +21,11 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
+
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.internal.managers.encryption.GridEncryptionManager;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 
 /**
@@ -260,7 +261,7 @@ public class EncryptedFileIO implements FileIO {
      * @param res Destination buffer.
      */
     private void storeCRC(ByteBuffer res) {
-        int crc = PureJavaCrc32.calcCrc32(res, encryptedDataSize());
+        int crc = FastCrc.calcCrc(res, encryptedDataSize());
 
         res.put((byte) (crc >> 24));
         res.put((byte) (crc >> 16));
@@ -274,7 +275,7 @@ public class EncryptedFileIO implements FileIO {
      * @param encrypted Encrypted data buffer.
      */
     private void checkCRC(ByteBuffer encrypted) throws IOException {
-        int crc = PureJavaCrc32.calcCrc32(encrypted, encryptedDataSize());
+        int crc = FastCrc.calcCrc(encrypted, encryptedDataSize());
 
         int storedCrc = 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
index 2e07867..fdf4705 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
@@ -27,6 +27,7 @@ import java.nio.file.Files;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -35,8 +36,8 @@ import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.processors.cache.persistence.AllocatedPageTracker;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static java.nio.file.StandardOpenOption.CREATE;
@@ -358,7 +359,7 @@ public class FilePageStore implements PageStore {
             pageBuf.position(0);
 
             if (!skipCrc) {
-                int curCrc32 = PureJavaCrc32.calcCrc32(pageBuf, pageSize);
+                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
 
                 if ((savedCrc32 ^ curCrc32) != 0)
                     throw new IgniteDataIntegrityViolationException("Failed to read page (CRC validation failed) " +
@@ -613,7 +614,7 @@ public class FilePageStore implements PageStore {
         try {
             pageBuf.position(0);
 
-            return PureJavaCrc32.calcCrc32(pageBuf, pageSize);
+            return FastCrc.calcCrc(pageBuf, pageSize);
         }
         finally {
             pageBuf.position(0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 5c6502b..8198149 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -98,8 +98,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStor
 import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.wal.aware.SegmentAware;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.LockedSegmentFileInputFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SegmentFileInputFactory;
@@ -2438,7 +2438,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             buf.position(0);
 
             // This call will move buffer position to the end of the record again.
-            int crcVal = PureJavaCrc32.calcCrc32(buf, curPos);
+            int crcVal = FastCrc.calcCrc(buf, curPos);
 
             buf.putInt(crcVal);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
index 7bfd10b..917640d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
@@ -91,8 +91,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SegmentFileInputFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SegmentIO;
@@ -2339,7 +2339,7 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
             buf.position(0);
 
             // This call will move buffer position to the end of the record again.
-            int crcVal = PureJavaCrc32.calcCrc32(buf, curPos);
+            int crcVal = FastCrc.calcCrc(buf, curPos);
 
             buf.putInt(crcVal);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java
new file mode 100644
index 0000000..0dcbafd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java
@@ -0,0 +1,101 @@
+/*
+ * 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.cache.persistence.wal.crc;
+
+import java.nio.ByteBuffer;
+import java.util.zip.CRC32;
+
+/**
+ * This CRC calculation implementation workf much faster then {@link PureJavaCrc32}
+ */
+public final class FastCrc {
+    /** CRC algo. */
+    private static final ThreadLocal<CRC32> CRC = ThreadLocal.withInitial(CRC32::new);
+
+    /** */
+    private final CRC32 crc = new CRC32();
+
+    /**
+     * Current value.
+     */
+    private int val;
+
+    /** */
+    public FastCrc() {
+        reset();
+    }
+
+    /**
+     * Preparation for further calculations.
+     */
+    public void reset() {
+        val = 0xffffffff;
+
+        crc.reset();
+    }
+
+    /**
+     * @return crc value.
+     */
+    public int getValue() {
+        return val;
+    }
+
+    /**
+     * @param buf Input buffer.
+     * @param len Data length.
+     */
+    public void update(final ByteBuffer buf, final int len) {
+        val = calcCrc(crc, buf, len);
+    }
+
+    /**
+     * @param buf Input buffer.
+     * @param len Data length.
+     *
+     * @return Crc checksum.
+     */
+    public static int calcCrc(ByteBuffer buf, int len) {
+        CRC32 crcAlgo = CRC.get();
+
+        int res = calcCrc(crcAlgo, buf, len);
+
+        crcAlgo.reset();
+
+        return res;
+    }
+
+    /**
+     * @param crcAlgo CRC algorithm.
+     * @param buf Input buffer.
+     * @param len Buffer length.
+     *
+     * @return Crc checksum.
+     */
+    private static int calcCrc(CRC32 crcAlgo, ByteBuffer buf, int len) {
+        int initLimit = buf.limit();
+
+        buf.limit(buf.position() + len);
+
+        crcAlgo.update(buf);
+
+        buf.limit(initLimit);
+
+        return (int)crcAlgo.getValue() ^ 0xFFFFFFFF;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/PureJavaCrc32.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/PureJavaCrc32.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/PureJavaCrc32.java
index 5e89a40..3f348d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/PureJavaCrc32.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/PureJavaCrc32.java
@@ -29,7 +29,9 @@ import java.nio.ByteBuffer;
  * succession.
  *
  * The current version is ~10x to 1.8x as fast as Sun's native java.util.zip.CRC32 in Java 1.6
+ * @deprecated Use {@link FastCrc} instead.
  */
+@Deprecated
 public class PureJavaCrc32 {
     /**
      * the current CRC value

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/FileInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/FileInput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/FileInput.java
index d19d17b..c9615f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/FileInput.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/FileInput.java
@@ -19,10 +19,12 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.io;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.zip.CRC32;
+
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -56,7 +58,7 @@ public interface FileInput extends ByteBufferBackedDataInput {
      */
     public class Crc32CheckingFileInput implements ByteBufferBackedDataInput, AutoCloseable {
         /** */
-        private final PureJavaCrc32 crc32 = new PureJavaCrc32();
+        private final FastCrc crc = new FastCrc();
 
         /** Last calc position. */
         private int lastCalcPosition;
@@ -93,7 +95,7 @@ public interface FileInput extends ByteBufferBackedDataInput {
         @Override public void close() throws Exception {
             updateCrc();
 
-            int val = crc32.getValue();
+            int val = crc.getValue();
 
             int writtenCrc =  this.readInt();
 
@@ -118,7 +120,7 @@ public interface FileInput extends ByteBufferBackedDataInput {
 
             buffer().position(lastCalcPosition);
 
-            crc32.update(delegate.buffer(), oldPos - lastCalcPosition);
+            crc.update(delegate.buffer(), oldPos - lastCalcPosition);
 
             lastCalcPosition = oldPos;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/SimpleFileInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/SimpleFileInput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/SimpleFileInput.java
index 5918b0b..1a1562e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/SimpleFileInput.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/io/SimpleFileInput.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.io;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander;
 import org.jetbrains.annotations.NotNull;
@@ -264,7 +265,7 @@ public class SimpleFileInput implements FileInput {
 
     /**
      * @param skipCheck If CRC check should be skipped.
-     * @return autoclosable fileInput, after its closing crc32 will be calculated and compared with saved one
+     * @return autoclosable fileInput, after its closing crc will be calculated and compared with saved one
      */
     public Crc32CheckingFileInput startRead(boolean skipCheck) {
         return new Crc32CheckingFileInput(this, skipCheck);

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
index c65f37c..31efa1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
@@ -22,6 +22,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
+
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
@@ -32,6 +33,7 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SegmentFileInputFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
@@ -39,7 +41,6 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofExc
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SegmentIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SimpleFileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.io.RecordIO;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -421,7 +422,7 @@ public class RecordV1Serializer implements RecordSerializer {
             buf.position(startPos);
 
             // This call will move buffer position to the end of the record again.
-            int crcVal = PureJavaCrc32.calcCrc32(buf, curPos - startPos);
+            int crcVal = FastCrc.calcCrc(buf, curPos - startPos);
 
             buf.putInt(crcVal);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
index 59dd3b7..3ad4c90 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
@@ -24,13 +24,14 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.io.SimpleFileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
-import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 
 /**
  *
@@ -59,6 +60,7 @@ public class IgniteDataIntegrityTests extends TestCase {
         );
 
         ByteBuffer buf = ByteBuffer.allocate(1024);
+
         ThreadLocalRandom curr = ThreadLocalRandom.current();
 
         for (int i = 0; i < 1024; i+=16) {
@@ -66,7 +68,7 @@ public class IgniteDataIntegrityTests extends TestCase {
             buf.putInt(curr.nextInt());
             buf.putInt(curr.nextInt());
             buf.position(i);
-            buf.putInt(PureJavaCrc32.calcCrc32(buf, 12));
+            buf.putInt(FastCrc.calcCrc(buf, 12));
         }
 
         buf.rewind();

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgnitePureJavaCrcCompatibility.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgnitePureJavaCrcCompatibility.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgnitePureJavaCrcCompatibility.java
new file mode 100644
index 0000000..faafad2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgnitePureJavaCrcCompatibility.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cache.persistence.db.wal.crc;
+
+import junit.framework.TestCase;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * PureJavaCrc32 previous crc algo realization vs java.util.zip.crc32 test.
+ */
+public class IgnitePureJavaCrcCompatibility extends TestCase {
+    /**
+     * Test crc algo equality results.
+     * @throws Exception
+     */
+    public void testAlgoEqual() throws Exception {
+        ByteBuffer buf = ByteBuffer.allocate(1024);
+
+        ThreadLocalRandom curr = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 1024; i+=16) {
+            buf.putInt(curr.nextInt());
+            buf.putInt(curr.nextInt());
+            buf.putInt(curr.nextInt());
+            buf.position(i);
+
+            buf.position(i);
+            int crc0 = FastCrc.calcCrc(buf, 12);
+
+            buf.position(i);
+            int crc1 = PureJavaCrc32.calcCrc32(buf, 12);
+
+            assertEquals(crc0, crc1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/091ace90/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index bf83bf5..ede5cab 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.Ignite
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteFsyncReplayWalIteratorInvalidCrcTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteReplayWalIteratorInvalidCrcTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteStandaloneWalIteratorInvalidCrcTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgnitePureJavaCrcCompatibility;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.reader.IgniteWalReaderTest;
 import org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneWalRecordsIteratorTest;
 
@@ -80,6 +81,7 @@ public class IgnitePdsTestSuite2 extends TestSuite {
         suite.addTestSuite(IgniteStandaloneWalIteratorInvalidCrcTest.class);
         suite.addTestSuite(IgniteReplayWalIteratorInvalidCrcTest.class);
         suite.addTestSuite(IgniteFsyncReplayWalIteratorInvalidCrcTest.class);
+        suite.addTestSuite(IgnitePureJavaCrcCompatibility.class);
 
         addRealPageStoreTests(suite);
 


[24/50] [abbrv] ignite git commit: IGNITE-9932 Ignoring exchanger critical section begin/end if called from illegal thread. - Fixes #5026.

Posted by sb...@apache.org.
IGNITE-9932 Ignoring exchanger critical section begin/end if called from illegal thread. - Fixes #5026.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/f156631c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f156631c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f156631c

Branch: refs/heads/ignite-gg-14206
Commit: f156631cdd4c1b69327f76df744b27d03f575b5c
Parents: 3bb0344
Author: Andrey Kuznetsov <st...@gmail.com>
Authored: Mon Oct 22 10:33:09 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 10:33:09 2018 +0300

----------------------------------------------------------------------
 .../cache/GridCachePartitionExchangeManager.java       | 13 ++++++++-----
 1 file changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f156631c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 0b8dd75..8b8efa3 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
@@ -2348,9 +2348,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * Should be called from exchange worker thread.
      */
     public void exchangerBlockingSectionBegin() {
-        assert exchWorker != null && Thread.currentThread() == exchWorker.runner();
-
-        exchWorker.blockingSectionBegin();
+        if (currentThreadIsExchanger())
+            exchWorker.blockingSectionBegin();
     }
 
     /**
@@ -2358,9 +2357,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * Should be called from exchange worker thread.
      */
     public void exchangerBlockingSectionEnd() {
-        assert exchWorker != null && Thread.currentThread() == exchWorker.runner();
+        if (currentThreadIsExchanger())
+            exchWorker.blockingSectionEnd();
+    }
 
-        exchWorker.blockingSectionEnd();
+    /** */
+    private boolean currentThreadIsExchanger() {
+        return exchWorker != null && Thread.currentThread() == exchWorker.runner();
     }
 
     /**


[15/50] [abbrv] ignite git commit: IGNITE-9082 Throwing checked exception during tx commit without node stopping leads to data corruption - Fixes #4809.

Posted by sb...@apache.org.
IGNITE-9082 Throwing checked exception during tx commit without node stopping leads to data corruption - Fixes #4809.

Signed-off-by: Ivan Rakov <ir...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5eb871e1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5eb871e1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5eb871e1

Branch: refs/heads/ignite-gg-14206
Commit: 5eb871e191a14fc21f6e2c62bdfa742e27c14695
Parents: 829dc1f
Author: Aleksei Scherbakov <al...@gmail.com>
Authored: Thu Oct 18 14:52:34 2018 +0300
Committer: Ivan Rakov <ir...@apache.org>
Committed: Thu Oct 18 14:52:34 2018 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |   5 -
 .../cache/GridCacheSharedContext.java           |   9 +
 .../GridDistributedTxRemoteAdapter.java         | 535 +++++++++----------
 .../distributed/dht/GridDhtTxFinishFuture.java  |  11 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  10 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   3 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |  59 +-
 .../near/GridNearTxFinishFuture.java            |  38 +-
 .../cache/distributed/near/GridNearTxLocal.java |   2 +-
 .../cache/transactions/IgniteTxAdapter.java     |  31 ++
 .../cache/transactions/IgniteTxHandler.java     | 119 ++---
 .../transactions/IgniteTxLocalAdapter.java      | 519 +++++++++---------
 .../processors/failure/FailureProcessor.java    |   8 +
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   1 -
 .../cache/GridCacheAbstractSelfTest.java        |   9 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |  66 +--
 .../cache/query/IndexingSpiQueryTxSelfTest.java |  74 +--
 .../AbstractTransactionIntergrityTest.java      | 111 ++--
 ...IntegrityWithPrimaryIndexCorruptionTest.java | 268 ++++++----
 ...ctionIntegrityWithSystemWorkerDeathTest.java |   6 +-
 .../TxDataConsistencyOnCommitFailureTest.java   | 234 ++++++++
 .../junits/common/GridCommonAbstractTest.java   |  19 +-
 .../testsuites/IgniteCacheTestSuite9.java       |   3 +
 23 files changed, 1220 insertions(+), 920 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
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 9bb8aec..ab5b725 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
@@ -2906,11 +2906,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             ver = newVer;
             flags &= ~IS_EVICT_DISABLED;
 
-            if (cctx.mvccEnabled())
-                cctx.offheap().mvccRemoveAll(this);
-            else
-                removeValue();
-
             onInvalidate();
 
             return obsoleteVersionExtras() != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 52d8525..b5cd82b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -1141,4 +1141,13 @@ public class GridCacheSharedContext<K, V> {
     public void readOnlyMode(boolean readOnlyMode) {
         this.readOnlyMode = readOnlyMode;
     }
+
+    /**
+     * For test purposes.
+     * @param txMgr Tx manager.
+     */
+    public void setTxManager(IgniteTxManager txMgr) {
+        this.txMgr = txMgr;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 7313197..4db4685 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -33,7 +33,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.InvalidEnvironmentException;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
@@ -55,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpda
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
-import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
@@ -512,312 +510,267 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
 
                         batchStoreCommit(writeMap().values());
 
-                        try {
-                            // Node that for near transactions we grab all entries.
-                            for (IgniteTxEntry txEntry : entries) {
-                                GridCacheContext cacheCtx = txEntry.context();
+                        // Node that for near transactions we grab all entries.
+                        for (IgniteTxEntry txEntry : entries) {
+                            GridCacheContext cacheCtx = txEntry.context();
 
-                                boolean replicate = cacheCtx.isDrEnabled();
+                            boolean replicate = cacheCtx.isDrEnabled();
 
+                            while (true) {
                                 try {
-                                    while (true) {
-                                        try {
-                                            GridCacheEntryEx cached = txEntry.cached();
+                                    GridCacheEntryEx cached = txEntry.cached();
 
-                                            if (cached == null)
-                                                txEntry.cached(cached = cacheCtx.cache().entryEx(txEntry.key(), topologyVersion()));
+                                    if (cached == null)
+                                        txEntry.cached(cached = cacheCtx.cache().entryEx(txEntry.key(), topologyVersion()));
 
-                                            if (near() && cacheCtx.dr().receiveEnabled()) {
-                                                cached.markObsolete(xidVer);
+                                    if (near() && cacheCtx.dr().receiveEnabled()) {
+                                        cached.markObsolete(xidVer);
 
-                                                break;
-                                            }
+                                        break;
+                                    }
 
-                                            GridNearCacheEntry nearCached = null;
+                                    GridNearCacheEntry nearCached = null;
 
-                                            if (updateNearCache(cacheCtx, txEntry.key(), topVer))
-                                                nearCached = cacheCtx.dht().near().peekExx(txEntry.key());
+                                    if (updateNearCache(cacheCtx, txEntry.key(), topVer))
+                                        nearCached = cacheCtx.dht().near().peekExx(txEntry.key());
 
-                                            if (!F.isEmpty(txEntry.entryProcessors()))
-                                                txEntry.cached().unswap(false);
+                                    if (!F.isEmpty(txEntry.entryProcessors()))
+                                        txEntry.cached().unswap(false);
 
-                                            IgniteBiTuple<GridCacheOperation, CacheObject> res =
-                                                applyTransformClosures(txEntry, false, ret);
+                                    IgniteBiTuple<GridCacheOperation, CacheObject> res =
+                                        applyTransformClosures(txEntry, false, ret);
 
-                                            GridCacheOperation op = res.get1();
-                                            CacheObject val = res.get2();
+                                    GridCacheOperation op = res.get1();
+                                    CacheObject val = res.get2();
 
-                                            GridCacheVersion explicitVer = txEntry.conflictVersion();
+                                    GridCacheVersion explicitVer = txEntry.conflictVersion();
 
-                                            if (explicitVer == null)
-                                                explicitVer = writeVersion();
+                                    if (explicitVer == null)
+                                        explicitVer = writeVersion();
 
-                                            if (txEntry.ttl() == CU.TTL_ZERO)
-                                                op = DELETE;
+                                    if (txEntry.ttl() == CU.TTL_ZERO)
+                                        op = DELETE;
 
-                                            boolean conflictNeedResolve = cacheCtx.conflictNeedResolve();
+                                    boolean conflictNeedResolve = cacheCtx.conflictNeedResolve();
 
-                                            GridCacheVersionConflictContext conflictCtx = null;
+                                    GridCacheVersionConflictContext conflictCtx = null;
 
-                                            if (conflictNeedResolve) {
-                                                IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContext>
-                                                    drRes = conflictResolve(op, txEntry, val, explicitVer, cached);
+                                    if (conflictNeedResolve) {
+                                        IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContext>
+                                            drRes = conflictResolve(op, txEntry, val, explicitVer, cached);
 
-                                                assert drRes != null;
+                                        assert drRes != null;
 
-                                                conflictCtx = drRes.get2();
+                                        conflictCtx = drRes.get2();
 
-                                                if (conflictCtx.isUseOld())
-                                                    op = NOOP;
-                                                else if (conflictCtx.isUseNew()) {
-                                                    txEntry.ttl(conflictCtx.ttl());
-                                                    txEntry.conflictExpireTime(conflictCtx.expireTime());
-                                                }
-                                                else if (conflictCtx.isMerge()) {
-                                                    op = drRes.get1();
-                                                    val = txEntry.context().toCacheObject(conflictCtx.mergeValue());
-                                                    explicitVer = writeVersion();
+                                        if (conflictCtx.isUseOld())
+                                            op = NOOP;
+                                        else if (conflictCtx.isUseNew()) {
+                                            txEntry.ttl(conflictCtx.ttl());
+                                            txEntry.conflictExpireTime(conflictCtx.expireTime());
+                                        }
+                                        else if (conflictCtx.isMerge()) {
+                                            op = drRes.get1();
+                                            val = txEntry.context().toCacheObject(conflictCtx.mergeValue());
+                                            explicitVer = writeVersion();
 
-                                                    txEntry.ttl(conflictCtx.ttl());
-                                                    txEntry.conflictExpireTime(conflictCtx.expireTime());
-                                                }
-                                            }
-                                            else
-                                                // Nullify explicit version so that innerSet/innerRemove will work as usual.
-                                                explicitVer = null;
-
-                                            GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null;
-
-                                            if (!near() && cacheCtx.group().persistenceEnabled() && cacheCtx.group().walEnabled() &&
-                                                op != NOOP && op != RELOAD && (op != READ || cctx.snapshot().needTxReadLogging())) {
-                                                if (dataEntries == null)
-                                                    dataEntries = new ArrayList<>(entries.size());
-
-                                                dataEntries.add(
-                                                        new T2<>(
-                                                                new DataEntry(
-                                                                        cacheCtx.cacheId(),
-                                                                        txEntry.key(),
-                                                                        val,
-                                                                        op,
-                                                                        nearXidVersion(),
-                                                                        writeVersion(),
-                                                                        0,
-                                                                        txEntry.key().partition(),
-                                                                        txEntry.updateCounter()
-                                                                ),
-                                                                txEntry
-                                                        )
-                                                );
-                                            }
+                                            txEntry.ttl(conflictCtx.ttl());
+                                            txEntry.conflictExpireTime(conflictCtx.expireTime());
+                                        }
+                                    }
+                                    else
+                                        // Nullify explicit version so that innerSet/innerRemove will work as usual.
+                                        explicitVer = null;
+
+                                    GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null;
+
+                                    if (!near() && cacheCtx.group().persistenceEnabled() && cacheCtx.group().walEnabled() &&
+                                        op != NOOP && op != RELOAD && (op != READ || cctx.snapshot().needTxReadLogging())) {
+                                        if (dataEntries == null)
+                                            dataEntries = new ArrayList<>(entries.size());
+
+                                        dataEntries.add(
+                                            new T2<>(
+                                                new DataEntry(
+                                                    cacheCtx.cacheId(),
+                                                    txEntry.key(),
+                                                    val,
+                                                    op,
+                                                    nearXidVersion(),
+                                                    writeVersion(),
+                                                    0,
+                                                    txEntry.key().partition(),
+                                                    txEntry.updateCounter()
+                                                ),
+                                                txEntry
+                                            )
+                                        );
+                                    }
 
-                                            if (op == CREATE || op == UPDATE) {
-                                                // Invalidate only for near nodes (backups cannot be invalidated).
-                                                if (isSystemInvalidate() || (isInvalidate() && cacheCtx.isNear()))
-                                                    cached.innerRemove(this,
-                                                        eventNodeId(),
-                                                        nodeId,
-                                                        false,
-                                                        true,
-                                                        true,
-                                                        txEntry.keepBinary(),
-                                                        txEntry.hasOldValue(),
-                                                        txEntry.oldValue(),
-                                                        topVer,
-                                                        null,
-                                                        replicate ? DR_BACKUP : DR_NONE,
-                                                        near() ? null : explicitVer,
-                                                        CU.subjectId(this, cctx),
-                                                        resolveTaskName(),
-                                                        dhtVer,
-                                                        txEntry.updateCounter(),
-                                                        mvccSnapshot());
-                                                else {
-                                                    assert val != null : txEntry;
-
-                                                    GridCacheUpdateTxResult updRes = cached.innerSet(this,
-                                                        eventNodeId(),
-                                                        nodeId,
-                                                        val,
-                                                        false,
-                                                        false,
-                                                        txEntry.ttl(),
-                                                        true,
-                                                        true,
-                                                        txEntry.keepBinary(),
-                                                        txEntry.hasOldValue(),
-                                                        txEntry.oldValue(),
-                                                        topVer,
-                                                        null,
-                                                        replicate ? DR_BACKUP : DR_NONE,
-                                                        txEntry.conflictExpireTime(),
-                                                        near() ? null : explicitVer,
-                                                        CU.subjectId(this, cctx),
-                                                        resolveTaskName(),
-                                                        dhtVer,
-                                                        txEntry.updateCounter(),
-                                                        mvccSnapshot());
-
-                                                    txEntry.updateCounter(updRes.updateCounter());
-
-                                                    if (updRes.loggedPointer() != null)
-                                                        ptr = updRes.loggedPointer();
-
-                                                    // Keep near entry up to date.
-                                                    if (nearCached != null) {
-                                                        CacheObject val0 = cached.valueBytes();
-
-                                                        nearCached.updateOrEvict(xidVer,
-                                                            val0,
-                                                            cached.expireTime(),
-                                                            cached.ttl(),
-                                                            nodeId,
-                                                            topVer);
-                                                    }
-                                                }
-                                            }
-                                            else if (op == DELETE) {
-                                                GridCacheUpdateTxResult updRes = cached.innerRemove(this,
-                                                    eventNodeId(),
+                                    if (op == CREATE || op == UPDATE) {
+                                        // Invalidate only for near nodes (backups cannot be invalidated).
+                                        if (isSystemInvalidate() || (isInvalidate() && cacheCtx.isNear()))
+                                            cached.innerRemove(this,
+                                                eventNodeId(),
+                                                nodeId,
+                                                false,
+                                                true,
+                                                true,
+                                                txEntry.keepBinary(),
+                                                txEntry.hasOldValue(),
+                                                txEntry.oldValue(),
+                                                topVer,
+                                                null,
+                                                replicate ? DR_BACKUP : DR_NONE,
+                                                near() ? null : explicitVer,
+                                                CU.subjectId(this, cctx),
+                                                resolveTaskName(),
+                                                dhtVer,
+                                                txEntry.updateCounter(),
+                                                mvccSnapshot());
+                                        else {
+                                            assert val != null : txEntry;
+
+                                            GridCacheUpdateTxResult updRes = cached.innerSet(this,
+                                                eventNodeId(),
+                                                nodeId,
+                                                val,
+                                                false,
+                                                false,
+                                                txEntry.ttl(),
+                                                true,
+                                                true,
+                                                txEntry.keepBinary(),
+                                                txEntry.hasOldValue(),
+                                                txEntry.oldValue(),
+                                                topVer,
+                                                null,
+                                                replicate ? DR_BACKUP : DR_NONE,
+                                                txEntry.conflictExpireTime(),
+                                                near() ? null : explicitVer,
+                                                CU.subjectId(this, cctx),
+                                                resolveTaskName(),
+                                                dhtVer,
+                                                txEntry.updateCounter(),
+                                                mvccSnapshot());
+
+                                            txEntry.updateCounter(updRes.updateCounter());
+
+                                            if (updRes.loggedPointer() != null)
+                                                ptr = updRes.loggedPointer();
+
+                                            // Keep near entry up to date.
+                                            if (nearCached != null) {
+                                                CacheObject val0 = cached.valueBytes();
+
+                                                nearCached.updateOrEvict(xidVer,
+                                                    val0,
+                                                    cached.expireTime(),
+                                                    cached.ttl(),
                                                     nodeId,
-                                                    false,
-                                                    true,
-                                                    true,
-                                                    txEntry.keepBinary(),
-                                                    txEntry.hasOldValue(),
-                                                    txEntry.oldValue(),
-                                                    topVer,
-                                                    null,
-                                                    replicate ? DR_BACKUP : DR_NONE,
-                                                    near() ? null : explicitVer,
-                                                    CU.subjectId(this, cctx),
-                                                    resolveTaskName(),
-                                                    dhtVer,
-                                                    txEntry.updateCounter(),
-                                                    mvccSnapshot());
-
-                                                txEntry.updateCounter(updRes.updateCounter());
-
-                                                if (updRes.loggedPointer() != null)
-                                                    ptr = updRes.loggedPointer();
-
-                                                // Keep near entry up to date.
-                                                if (nearCached != null)
-                                                    nearCached.updateOrEvict(xidVer, null, 0, 0, nodeId, topVer);
-                                            }
-                                            else if (op == RELOAD) {
-                                                CacheObject reloaded = cached.innerReload();
-
-                                                if (nearCached != null) {
-                                                    nearCached.innerReload();
-
-                                                    nearCached.updateOrEvict(cached.version(),
-                                                        reloaded,
-                                                        cached.expireTime(),
-                                                        cached.ttl(),
-                                                        nodeId,
-                                                        topVer);
-                                                }
+                                                    topVer);
                                             }
-                                            else if (op == READ) {
-                                                assert near();
-
-                                                if (log.isDebugEnabled())
-                                                    log.debug("Ignoring READ entry when committing: " + txEntry);
-                                            }
-                                            // No-op.
-                                            else {
-                                                if (conflictCtx == null || !conflictCtx.isUseOld()) {
-                                                    if (txEntry.ttl() != CU.TTL_NOT_CHANGED)
-                                                        cached.updateTtl(null, txEntry.ttl());
-
-                                                    if (nearCached != null) {
-                                                        CacheObject val0 = cached.valueBytes();
-
-                                                        nearCached.updateOrEvict(xidVer,
-                                                            val0,
-                                                            cached.expireTime(),
-                                                            cached.ttl(),
-                                                            nodeId,
-                                                            topVer);
-                                                    }
-                                                }
-                                            }
-
-                                            // Assert after setting values as we want to make sure
-                                            // that if we replaced removed entries.
-                                            assert
-                                                txEntry.op() == READ || onePhaseCommit() ||
-                                                    // If candidate is not there, then lock was explicit
-                                                    // and we simply allow the commit to proceed.
-                                                    !cached.hasLockCandidateUnsafe(xidVer) || cached.lockedByUnsafe(xidVer) :
-                                                "Transaction does not own lock for commit [entry=" + cached +
-                                                    ", tx=" + this + ']';
-
-                                            // Break out of while loop.
-                                            break;
-                                        }
-                                        catch (GridCacheEntryRemovedException ignored) {
-                                            if (log.isDebugEnabled())
-                                                log.debug("Attempting to commit a removed entry (will retry): " + txEntry);
-
-                                            // Renew cached entry.
-                                            txEntry.cached(cacheCtx.cache().entryEx(txEntry.key(), topologyVersion()));
                                         }
                                     }
-                                }
-                                catch (Throwable ex) {
-                                    boolean isNodeStopping = X.hasCause(ex, NodeStoppingException.class);
-                                    boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class);
-
-                                    // In case of error, we still make the best effort to commit,
-                                    // as there is no way to rollback at this point.
-                                    err = new IgniteTxHeuristicCheckedException("Commit produced a runtime exception " +
-                                        "(all transaction entries will be invalidated): " + CU.txString(this), ex);
-
-                                    if (isNodeStopping) {
-                                        U.warn(log, "Failed to commit transaction, node is stopping [tx=" + this +
-                                            ", err=" + ex + ']');
-                                    }
-                                    else if (hasInvalidEnvironmentIssue) {
-                                        U.warn(log, "Failed to commit transaction, node is in invalid state and will be stopped [tx=" + this +
-                                            ", err=" + ex + ']');
+                                    else if (op == DELETE) {
+                                        GridCacheUpdateTxResult updRes = cached.innerRemove(this,
+                                            eventNodeId(),
+                                            nodeId,
+                                            false,
+                                            true,
+                                            true,
+                                            txEntry.keepBinary(),
+                                            txEntry.hasOldValue(),
+                                            txEntry.oldValue(),
+                                            topVer,
+                                            null,
+                                            replicate ? DR_BACKUP : DR_NONE,
+                                            near() ? null : explicitVer,
+                                            CU.subjectId(this, cctx),
+                                            resolveTaskName(),
+                                            dhtVer,
+                                            txEntry.updateCounter(),
+                                            mvccSnapshot());
+
+                                        txEntry.updateCounter(updRes.updateCounter());
+
+                                        if (updRes.loggedPointer() != null)
+                                            ptr = updRes.loggedPointer();
+
+                                        // Keep near entry up to date.
+                                        if (nearCached != null)
+                                            nearCached.updateOrEvict(xidVer, null, 0, 0, nodeId, topVer);
                                     }
-                                    else
-                                        U.error(log, "Commit failed.", err);
-
-                                    state(UNKNOWN);
-
-                                    if (hasInvalidEnvironmentIssue)
-                                        cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, ex));
-                                    else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or invalidation.
-                                        try {
-                                            // Courtesy to minimize damage.
-                                            uncommit();
+                                    else if (op == RELOAD) {
+                                        CacheObject reloaded = cached.innerReload();
+
+                                        if (nearCached != null) {
+                                            nearCached.innerReload();
+
+                                            nearCached.updateOrEvict(cached.version(),
+                                                reloaded,
+                                                cached.expireTime(),
+                                                cached.ttl(),
+                                                nodeId,
+                                                topVer);
                                         }
-                                        catch (Throwable ex1) {
-                                            U.error(log, "Failed to uncommit transaction: " + this, ex1);
+                                    }
+                                    else if (op == READ) {
+                                        assert near();
 
-                                            if (ex1 instanceof Error)
-                                                throw ex1;
+                                        if (log.isDebugEnabled())
+                                            log.debug("Ignoring READ entry when committing: " + txEntry);
+                                    }
+                                    // No-op.
+                                    else {
+                                        if (conflictCtx == null || !conflictCtx.isUseOld()) {
+                                            if (txEntry.ttl() != CU.TTL_NOT_CHANGED)
+                                                cached.updateTtl(null, txEntry.ttl());
+
+                                            if (nearCached != null) {
+                                                CacheObject val0 = cached.valueBytes();
+
+                                                nearCached.updateOrEvict(xidVer,
+                                                    val0,
+                                                    cached.expireTime(),
+                                                    cached.ttl(),
+                                                    nodeId,
+                                                    topVer);
+                                            }
                                         }
                                     }
 
-                                    if (ex instanceof Error)
-                                        throw (Error) ex;
+                                    // Assert after setting values as we want to make sure
+                                    // that if we replaced removed entries.
+                                    assert
+                                        txEntry.op() == READ || onePhaseCommit() ||
+                                            // If candidate is not there, then lock was explicit
+                                            // and we simply allow the commit to proceed.
+                                            !cached.hasLockCandidateUnsafe(xidVer) || cached.lockedByUnsafe(xidVer) :
+                                        "Transaction does not own lock for commit [entry=" + cached +
+                                            ", tx=" + this + ']';
+
+                                    // Break out of while loop.
+                                    break;
+                                }
+                                catch (GridCacheEntryRemovedException ignored) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Attempting to commit a removed entry (will retry): " + txEntry);
 
-                                    throw err;
+                                    // Renew cached entry.
+                                    txEntry.cached(cacheCtx.cache().entryEx(txEntry.key(), topologyVersion()));
                                 }
                             }
+                        }
 
-                            // Apply cache size deltas.
-                            applyTxSizes();
+                        // Apply cache size deltas.
+                        applyTxSizes();
 
-                            TxCounters txCntrs = txCounters(false);
+                        TxCounters txCntrs = txCounters(false);
 
-                            // Apply update counters.
-                            if (txCntrs != null)
-                                applyPartitionsUpdatesCounters(txCntrs.updateCounters());
+                        // Apply update counters.
+                        if (txCntrs != null)
+                            applyPartitionsUpdatesCounters(txCntrs.updateCounters());
 
                             cctx.mvccCaching().onTxFinished(this, true);
 
@@ -827,18 +780,32 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                         .map(tuple -> tuple.get1().partitionCounter(tuple.get2().updateCounter()))
                                         .collect(Collectors.toList());
 
-                                cctx.wal().log(new DataRecord(entriesWithCounters));
-                            }
+                            cctx.wal().log(new DataRecord(entriesWithCounters));
+                        }
+
+                        if (ptr != null && !cctx.tm().logTxRecords())
+                            cctx.wal().flush(ptr, false);
+                    }
+                    catch (Throwable ex) {
+                        state(UNKNOWN);
 
-                            if (ptr != null && !cctx.tm().logTxRecords())
-                                cctx.wal().flush(ptr, false);
+                        if (X.hasCause(ex, NodeStoppingException.class)) {
+                            U.warn(log, "Failed to commit transaction, node is stopping [tx=" + CU.txString(this) +
+                                ", err=" + ex + ']');
+
+                            return;
                         }
-                        catch (StorageException e) {
-                            err = e;
 
-                            throw new IgniteCheckedException("Failed to log transaction record " +
-                                "(transaction will be rolled back): " + this, e);
+                        err = heuristicException(ex);
+
+                        try {
+                            uncommit();
+                        }
+                        catch (Throwable e) {
+                            err.addSuppressed(e);
                         }
+
+                        throw err;
                     }
                     finally {
                         cctx.database().checkpointReadUnlock();
@@ -878,9 +845,19 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                 throw new IgniteCheckedException("Invalid transaction state for commit [state=" + state + ", tx=" + this + ']');
 
             rollbackRemoteTx();
+
+            return;
         }
 
-        commitIfLocked();
+        try {
+            commitIfLocked();
+        }
+        catch (IgniteTxHeuristicCheckedException e) {
+            // Treat heuristic exception as critical.
+            cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+
+            throw e;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
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 21eb7b2..9f96b46 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
@@ -173,10 +173,13 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
         if (ERR_UPD.compareAndSet(this, null, e)) {
             tx.setRollbackOnly();
 
-            if (X.hasCause(e, InvalidEnvironmentException.class, NodeStoppingException.class))
+            if (X.hasCause(e, NodeStoppingException.class) || cctx.kernalContext().failure().nodeStopping())
                 onComplete();
-            else
+            else {
+                // Rolling back a remote transaction may result in partial commit.
+                // This is only acceptable in tests with no-op failure handler.
                 finish(false);
+            }
         }
     }
 
@@ -230,9 +233,9 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
 
             if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) {
                 try {
-                    boolean hasInvalidEnvironmentIssue = X.hasCause(err, InvalidEnvironmentException.class, NodeStoppingException.class);
+                    boolean nodeStopping = X.hasCause(err, NodeStoppingException.class);
 
-                    this.tx.tmFinish(err == null, hasInvalidEnvironmentIssue, false);
+                    this.tx.tmFinish(err == null, nodeStopping || cctx.kernalContext().failure().nodeStopping(), false);
                 }
                 catch (IgniteCheckedException finishErr) {
                     U.error(log, "Failed to finish tx: " + tx, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index a091d44..ca451f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -24,6 +24,8 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -39,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPr
 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.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
@@ -46,6 +49,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringBuilder;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -467,7 +471,11 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
                     ", tx=" + CU.txString(this) + ']');
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to finish transaction [commit=" + commit + ", tx=" + this + ']', e);
+            logTxFinishErrorSafe(log, commit, e);
+
+            // Treat heuristic exception as critical.
+            if (X.hasCause(e, IgniteTxHeuristicCheckedException.class))
+                cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
 
             err = e;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
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 ffa383b..483990f 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
@@ -30,6 +30,8 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
@@ -43,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTx
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
 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;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
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 741faee..c505677 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
@@ -36,6 +36,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteDiagnosticAware;
 import org.apache.ignite.internal.IgniteDiagnosticPrepareContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -740,8 +742,6 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
 
             if (tx.commitOnPrepare()) {
                 if (tx.markFinalizing(IgniteInternalTx.FinalizationStatus.USER_FINISH)) {
-                    IgniteInternalFuture<IgniteInternalTx> fut = null;
-
                     CIX1<IgniteInternalFuture<IgniteInternalTx>> resClo =
                         new CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
                             @Override public void applyx(IgniteInternalFuture<IgniteInternalTx> fut) {
@@ -753,42 +753,43 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
                             }
                         };
 
-                    if (prepErr == null) {
-                        try {
-                            fut = tx.commitAsync();
-                        }
-                        catch (RuntimeException | Error e) {
-                            Exception hEx = new IgniteTxHeuristicCheckedException("Commit produced a runtime " +
-                                "exception: " + CU.txString(tx), e);
-
-                            res.error(hEx);
+                    try {
+                        if (prepErr == null) {
+                            try {
+                                tx.commitAsync().listen(resClo);
+                            }
+                            catch (Throwable e) {
+                                res.error(e);
 
-                            tx.systemInvalidate(true);
+                                tx.systemInvalidate(true);
 
-                            try {
-                                fut = tx.rollbackAsync();
+                                try {
+                                    tx.rollbackAsync().listen(resClo);
+                                }
+                                catch (Throwable e1) {
+                                    e.addSuppressed(e1);
+                                }
 
-                                fut.listen(resClo);
+                                throw e;
                             }
-                            catch (Throwable e1) {
-                                e.addSuppressed(e1);
+                        }
+                        else if (!cctx.kernalContext().isStopping()) {
+                            try {
+                                tx.rollbackAsync().listen(resClo);
                             }
+                            catch (Throwable e) {
+                                if (err != null)
+                                    err.addSuppressed(e);
 
-                            throw e;
+                                throw err;
+                            }
                         }
-
                     }
-                    else if (!cctx.kernalContext().isStopping())
-                        try {
-                            fut = tx.rollbackAsync();
-                        }
-                        catch (Throwable e) {
-                            err.addSuppressed(e);
-                            fut = null;
-                        }
+                    catch (Throwable e){
+                        tx.logTxFinishErrorSafe(log, true, e);
 
-                    if (fut != null)
-                        fut.listen(resClo);
+                        cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+                    }
                 }
             }
             else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
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 4a4d8e3..befa305 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
@@ -311,7 +311,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
             if (err != null) {
                 tx.setRollbackOnly();
 
-                nodeStop = err instanceof NodeStoppingException;
+                nodeStop = err instanceof NodeStoppingException || cctx.kernalContext().failure().nodeStopping();
             }
 
             if (commit) {
@@ -357,29 +357,6 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
                 }
 
                 if (super.onDone(tx0, err)) {
-                    if (error() instanceof IgniteTxHeuristicCheckedException && !nodeStop) {
-                        AffinityTopologyVersion topVer = tx.topologyVersion();
-
-                        for (IgniteTxEntry e : tx.writeMap().values()) {
-                            GridCacheContext cacheCtx = e.context();
-
-                            try {
-                                if (e.op() != NOOP && !cacheCtx.affinity().keyLocalNode(e.key(), topVer)) {
-                                    GridCacheEntryEx entry = cacheCtx.cache().peekEx(e.key());
-
-                                    if (entry != null)
-                                        entry.invalidate(tx.xidVersion());
-                                }
-                            }
-                            catch (Throwable t) {
-                                U.error(log, "Failed to invalidate entry.", t);
-
-                                if (t instanceof Error)
-                                    throw (Error)t;
-                            }
-                        }
-                    }
-
                     // Don't forget to clean up.
                     cctx.mvcc().removeFuture(futId);
 
@@ -402,8 +379,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
     }
 
     /** {@inheritDoc} */
-    @Override @SuppressWarnings("ForLoopReplaceableByForEach")
-    public void finish(final boolean commit, final boolean clearThreadMap, final boolean onTimeout) {
+    @Override public void finish(final boolean commit, final boolean clearThreadMap, final boolean onTimeout) {
         if (!cctx.mvcc().addFuture(this, futureId()))
             return;
 
@@ -490,18 +466,22 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
                     }
                 }
 
+                // Cleanup transaction if heuristic failure.
+                if (tx.state() == UNKNOWN)
+                    cctx.tm().rollbackTx(tx, clearThreadMap, false);
+
                 if ((tx.onePhaseCommit() && needFinishOnePhase(commit)) || (!tx.onePhaseCommit() && mappings != null)) {
                     if (mappings.single()) {
                         GridDistributedTxMapping mapping = mappings.singleMapping();
 
                         if (mapping != null) {
-                            assert !hasFutures() || waitTxs != null : futures();
+                            assert !hasFutures() || isDone() || waitTxs != null : futures();
 
                             finish(1, mapping, commit, !clearThreadMap);
                         }
                     }
                     else {
-                        assert !hasFutures() || waitTxs != null : futures();
+                        assert !hasFutures() || isDone() || waitTxs != null : futures();
 
                         finish(mappings.mappings(), commit, !clearThreadMap);
                     }
@@ -762,7 +742,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
     /**
      * @param mappings Mappings.
      * @param commit Commit flag.
-     * @param {@code true} If need to add completed version on finish.
+     * @param useCompletedVer {@code True} if need to add completed version on finish.
      */
     private void finish(Iterable<GridDistributedTxMapping> mappings, boolean commit, boolean useCompletedVer) {
         int miniId = 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
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 76d464e..f56d99b 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
@@ -3955,7 +3955,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
 
                                 assert rollbackFut.isDone() : rollbackFut;
                             }
-                            else
+                            else // First finish attempt was unsuccessful. Try again.
                                 rollbackFut.finish(false, clearThreadMap, onTimeout);
                         }
                         else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index b091061..0d3ba75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -75,6 +75,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
 import org.apache.ignite.internal.processors.cluster.BaselineTopology;
+import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.GridSetWrapper;
@@ -764,6 +765,36 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
             "[timeout=" + timeout() + ", tx=" + CU.txString(this) + ']');
     }
 
+    /**
+     * @param ex Root cause.
+     */
+    public final IgniteCheckedException heuristicException(Throwable ex) {
+        return new IgniteTxHeuristicCheckedException("Committing a transaction has produced runtime exception", ex);
+    }
+
+    /**
+     * @param log Log.
+     * @param commit Commit.
+     * @param e Exception.
+     */
+    public void logTxFinishErrorSafe(@Nullable IgniteLogger log, boolean commit, Throwable e) {
+        assert e != null : "Exception is expected";
+
+        final String fmt = "Failed completing the transaction: [commit=%s, tx=%s, plc=%s]";
+
+        try {
+            // First try printing a full transaction. This is error prone.
+            U.error(log, String.format(fmt, commit, this,
+                cctx.gridConfig().getFailureHandler().getClass().getSimpleName()), e);
+        }
+        catch (Throwable e0) {
+            e.addSuppressed(e0);
+
+            U.error(log, String.format(fmt, commit, CU.txString(this),
+                cctx.gridConfig().getFailureHandler().getClass().getSimpleName()), e);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public GridCacheVersion xidVersion() {
         return xidVer;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
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 4c7b65d..895a9d1 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
@@ -1030,45 +1030,34 @@ public class IgniteTxHandler {
         }
         catch (Throwable e) {
             try {
-                U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e);
-            }
-            catch (Throwable e0) {
-                ClusterNode node0 = ctx.discovery().node(nodeId);
-
-                U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" +
-                        CU.txString(tx) + ']', e);
-
-                U.error(log, "Failed to log message due to an error: ", e0);
+                if (tx != null) {
+                    tx.commitError(e);
 
-                if (node0 != null && (!node0.isClient() || node0.isLocal())) {
-                    ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+                    tx.systemInvalidate(true);
 
-                    throw e;
-                }
-            }
-
-            if (tx != null) {
-                tx.commitError(e);
-
-                tx.systemInvalidate(true);
+                    try {
+                        IgniteInternalFuture<IgniteInternalTx> res = tx.rollbackDhtLocalAsync();
 
-                try {
-                    IgniteInternalFuture<IgniteInternalTx> res = tx.rollbackDhtLocalAsync();
+                        // Only for error logging.
+                        res.listen(CU.errorLogger(log));
 
-                    // Only for error logging.
-                    res.listen(CU.errorLogger(log));
+                        return res;
+                    }
+                    catch (Throwable e1) {
+                        e.addSuppressed(e1);
+                    }
 
-                    return res;
+                    tx.logTxFinishErrorSafe(log, req.commit(), e);
                 }
-                catch (Throwable e1) {
-                    e.addSuppressed(e1);
-                }
-            }
 
-            if (e instanceof Error)
-                throw (Error)e;
+                if (e instanceof Error)
+                    throw (Error)e;
 
-            return new GridFinishedFuture<>(e);
+                return new GridFinishedFuture<>(e);
+            }
+            finally {
+                ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+            }
         }
     }
 
@@ -1093,20 +1082,26 @@ public class IgniteTxHandler {
                 return tx.rollbackAsyncLocal();
         }
         catch (Throwable e) {
-            U.error(log, "Failed completing transaction [commit=" + commit + ", tx=" + tx + ']', e);
-
-            if (e instanceof Error)
-                throw e;
+            try {
+                if (tx != null) {
+                    try {
+                        return tx.rollbackNearTxLocalAsync();
+                    }
+                    catch (Throwable e1) {
+                        e.addSuppressed(e1);
+                    }
 
-            if (tx != null)
-                try {
-                    return tx.rollbackNearTxLocalAsync();
-                }
-                catch (Throwable e1) {
-                    e.addSuppressed(e1);
+                    tx.logTxFinishErrorSafe(log, commit, e);
                 }
 
-            return new GridFinishedFuture<>(e);
+                if (e instanceof Error)
+                    throw e;
+
+                return new GridFinishedFuture<>(e);
+            }
+            finally {
+                ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+            }
         }
     }
 
@@ -1193,10 +1188,6 @@ public class IgniteTxHandler {
                 if (log.isDebugEnabled())
                     log.debug("Optimistic failure for remote transaction (will rollback): " + req);
             }
-            else if (e instanceof IgniteTxHeuristicCheckedException) {
-                U.warn(log, "Failed to commit transaction (all transaction entries were invalidated): " +
-                    CU.txString(dhtTx));
-            }
             else
                 U.error(log, "Failed to process prepare request: " + req, e);
 
@@ -1421,9 +1412,10 @@ public class IgniteTxHandler {
                 tx.rollbackRemoteTx();
             }
         }
+        catch (IgniteTxHeuristicCheckedException e) {
+            // Already uncommitted.
+        }
         catch (Throwable e) {
-            U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e);
-
             // Mark transaction for invalidate.
             tx.invalidate(true);
             tx.systemInvalidate(true);
@@ -1441,6 +1433,8 @@ public class IgniteTxHandler {
     }
 
     /**
+     * Finish for one-phase distributed tx.
+     *
      * @param tx Transaction.
      * @param req Request.
      */
@@ -1464,22 +1458,27 @@ public class IgniteTxHandler {
             throw e;
         }
         catch (Throwable e) {
-            U.error(log, "Failed committing transaction [tx=" + tx + ']', e);
+            try {
+                // Mark transaction for invalidate.
+                tx.invalidate(true);
 
-            // Mark transaction for invalidate.
-            tx.invalidate(true);
-            tx.systemInvalidate(true);
+                tx.systemInvalidate(true);
 
-            try {
-                tx.rollbackRemoteTx();
+                try {
+                    tx.rollbackRemoteTx();
+                }
+                catch (Throwable e1) {
+                    e.addSuppressed(e1);
+                }
+
+                tx.logTxFinishErrorSafe(log, true, e);
+
+                if (e instanceof Error)
+                    throw (Error)e;
             }
-            catch (Throwable e1) {
-                e.addSuppressed(e1);
-                U.error(log, "Failed to automatically rollback transaction: " + tx, e1);
+            finally {
+                ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
             }
-
-            if (e instanceof Error)
-                throw (Error)e;
         }
     }
 


[17/50] [abbrv] ignite git commit: IGNITE-9155 ExchangeFuture will not complete with Exception on state change failure - Fixes #4467.

Posted by sb...@apache.org.
IGNITE-9155 ExchangeFuture will not complete with Exception on state change failure - Fixes #4467.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/177d571b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/177d571b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/177d571b

Branch: refs/heads/ignite-gg-14206
Commit: 177d571b57f8dde2c82e2a7e4ba9b5d453f4d8df
Parents: a3b624d
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Thu Oct 18 19:34:55 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 18 19:34:55 2018 +0300

----------------------------------------------------------------------
 .../preloader/GridDhtPartitionsExchangeFuture.java   | 15 +++------------
 1 file changed, 3 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/177d571b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 0fe1a25..4b4b6df 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
@@ -3440,8 +3440,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             }
 
             if (stateChangeExchange()) {
-                IgniteCheckedException err = null;
-
                 StateChangeRequest req = exchActions.stateChangeRequest();
 
                 assert req != null : exchActions;
@@ -3451,8 +3449,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 if (!F.isEmpty(exchangeGlobalExceptions)) {
                     stateChangeErr = true;
 
-                    err = new IgniteCheckedException("Cluster state change failed.");
-
                     cctx.kernalContext().state().onStateChangeError(exchangeGlobalExceptions, req);
                 }
                 else {
@@ -3484,7 +3480,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 cctx.discovery().sendCustomEvent(stateFinishMsg);
 
                 if (!centralizedAff)
-                    onDone(exchCtx.events().topologyVersion(), err);
+                    onDone(exchCtx.events().topologyVersion(), null);
             }
         }
         catch (IgniteCheckedException e) {
@@ -3969,15 +3965,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             updatePartitionFullMap(resTopVer, msg);
 
-            IgniteCheckedException err = null;
-
-            if (stateChangeExchange() && !F.isEmpty(msg.getErrorsMap())) {
-                err = new IgniteCheckedException("Cluster state change failed");
-
+            if (stateChangeExchange() && !F.isEmpty(msg.getErrorsMap()))
                 cctx.kernalContext().state().onStateChangeError(msg.getErrorsMap(), exchActions.stateChangeRequest());
-            }
 
-            onDone(resTopVer, err);
+            onDone(resTopVer, null);
         }
         catch (IgniteCheckedException e) {
             onDone(e);


[26/50] [abbrv] ignite git commit: IGNITE-9837 Fixed flaky BinaryMetadataUpdatesFlowTest.testConcurrentMetadataUpdates - Fixes #4969.

Posted by sb...@apache.org.
IGNITE-9837 Fixed flaky BinaryMetadataUpdatesFlowTest.testConcurrentMetadataUpdates - Fixes #4969.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5fa5c548
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5fa5c548
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5fa5c548

Branch: refs/heads/ignite-gg-14206
Commit: 5fa5c5483d7dda59cc7806cd647f00bbdb66b216
Parents: 6f28742
Author: NSAmelchev <ns...@gmail.com>
Authored: Mon Oct 22 13:36:32 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 13:36:32 2018 +0300

----------------------------------------------------------------------
 .../processors/cache/binary/BinaryMetadataUpdatesFlowTest.java | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5fa5c548/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
index 7e8c086..eab7cfb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
@@ -310,6 +310,9 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
 
         killer.interrupt();
         resurrection.interrupt();
+
+        killer.join();
+        resurrection.join();
     }
 
     /**
@@ -351,6 +354,9 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
 
         killer.interrupt();
         resurrection.interrupt();
+
+        killer.join();
+        resurrection.join();
     }
 
     /**


[28/50] [abbrv] ignite git commit: IGNITE-9694 Add tests to check that reading queries are not blocked on exchange events that don't change data visibility - Fixes #4926.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnCreateDestoryIndexTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnCreateDestoryIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnCreateDestoryIndexTest.java
new file mode 100644
index 0000000..469ec93
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnCreateDestoryIndexTest.java
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.distributed;
+
+import java.lang.annotation.Annotation;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnReadAbstractTest.Params;
+import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class CacheBlockOnCreateDestoryIndexTest extends GridCommonAbstractTest {
+    /** */
+    private final List<? extends CacheBlockOnReadAbstractTest> tests = Arrays.asList(
+        new CacheBlockOnSingleGetTest() {
+            /** {@inheritDoc} */
+            @Nullable @Override protected <A extends Annotation> A currentTestAnnotation(Class<A> annotationCls) {
+                return CacheBlockOnCreateDestoryIndexTest.this.currentTestAnnotation(annotationCls);
+            }
+        },
+        new CacheBlockOnGetAllTest() {
+            /** {@inheritDoc} */
+            @Nullable @Override protected <A extends Annotation> A currentTestAnnotation(Class<A> annotationCls) {
+                return CacheBlockOnCreateDestoryIndexTest.this.currentTestAnnotation(annotationCls);
+            }
+        },
+        new CacheBlockOnScanTest() {
+            /** {@inheritDoc} */
+            @Nullable @Override protected <A extends Annotation> A currentTestAnnotation(Class<A> annotationCls) {
+                return CacheBlockOnCreateDestoryIndexTest.this.currentTestAnnotation(annotationCls);
+            }
+        },
+        new CacheBlockOnSqlQueryTest() {
+            /** {@inheritDoc} */
+            @Nullable @Override protected <A extends Annotation> A currentTestAnnotation(Class<A> annotationCls) {
+                return CacheBlockOnCreateDestoryIndexTest.this.currentTestAnnotation(annotationCls);
+            }
+        }
+    );
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        currentTest().beforeTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        currentTest().afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testCreateIndexAtomicPartitionedGet() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testCreateIndexAtomicReplicatedGet() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testCreateIndexTransactionalPartitionedGet() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testCreateIndexTransactionalReplicatedGet() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testCreateIndexAtomicPartitionedGetAll() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testCreateIndexAtomicReplicatedGetAll() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testCreateIndexTransactionalPartitionedGetAll() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testCreateIndexTransactionalReplicatedGetAll() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testCreateIndexAtomicPartitionedScan() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testCreateIndexAtomicReplicatedScan() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testCreateIndexTransactionalPartitionedScan() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testCreateIndexTransactionalReplicatedScan() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testCreateIndexAtomicPartitionedSqlQuery() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testCreateIndexAtomicReplicatedSqlQuery() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testCreateIndexTransactionalPartitionedSqlQuery() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testCreateIndexTransactionalReplicatedSqlQuery() throws Exception {
+        doTestCreateIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testDestroyIndexAtomicPartitionedGet() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testDestroyIndexAtomicReplicatedGet() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testDestroyIndexTransactionalPartitionedGet() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(0)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testDestroyIndexTransactionalReplicatedGet() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testDestroyIndexAtomicPartitionedGetAll() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testDestroyIndexAtomicReplicatedGetAll() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testDestroyIndexTransactionalPartitionedGetAll() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(1)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testDestroyIndexTransactionalReplicatedGetAll() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testDestroyIndexAtomicPartitionedScan() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testDestroyIndexAtomicReplicatedScan() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testDestroyIndexTransactionalPartitionedScan() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(2)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testDestroyIndexTransactionalReplicatedScan() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testDestroyIndexAtomicPartitionedSqlQuery() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testDestroyIndexAtomicReplicatedSqlQuery() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testDestroyIndexTransactionalPartitionedSqlQuery() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @TestIndex(3)
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testDestroyIndexTransactionalReplicatedSqlQuery() throws Exception {
+        doTestDestroyIndex();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestCreateIndex() throws Exception {
+        IgniteEx ignite = currentTest().baseline().get(0);
+
+        List<T3<String, String, String>> caches = createCaches(ignite);
+
+        currentTest().doTest(
+            msg -> msg instanceof SchemaOperationStatusMessage,
+            () -> createIndex(ignite, caches.remove(caches.size() - 1))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestDestroyIndex() throws Exception {
+        IgniteEx ignite = currentTest().baseline().get(0);
+
+        List<T3<String, String, String>> caches = createCaches(ignite);
+
+        for (T3<String, String, String> pair : caches)
+            createIndex(ignite, pair);
+
+        currentTest().doTest(
+            msg -> msg instanceof SchemaOperationStatusMessage,
+            () -> destroyIndex(ignite, caches.remove(caches.size() - 1))
+        );
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     * @return 3 pairs {@code {cacheName, tableName, indexName}} for further sql operations.
+     */
+    @NotNull private List<T3<String, String, String>> createCaches(IgniteEx ignite) {
+        List<T3<String, String, String>> caches = new ArrayList<>();
+
+        for (int i = 0; i < 3; i++) {
+            String tblName = "TABLE_" + UUID.randomUUID().toString().replace('-', '_');
+
+            String cacheName = "CACHE_" + tblName;
+
+            CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>(cacheName).setSqlSchema("PUBLIC");
+
+            IgniteCache<?, ?> cache = ignite.createCache(ccfg);
+
+            String createTblQryStr = String.format(
+                "CREATE TABLE %s (id LONG, name VARCHAR, city_id LONG, PRIMARY KEY (id, city_id)) " +
+                    "WITH \"backups=1, affinityKey=city_id\"",
+                tblName
+            );
+
+            cache.query(new SqlFieldsQuery(createTblQryStr)).getAll();
+
+            String idxName = "IDX_" + tblName;
+
+            caches.add(new T3<>(cacheName, tblName, idxName));
+        }
+
+        return caches;
+    }
+
+    /**
+     *
+     */
+    private void createIndex(IgniteEx ignite, T3<String, String, String> pair) {
+        IgniteCache<?, ?> cache = ignite.getOrCreateCache(pair.get1());
+
+        String createIdxQryStr = String.format("CREATE INDEX %S on %s (city_id)", pair.get3(), pair.get2());
+
+        cache.query(new SqlFieldsQuery(createIdxQryStr)).getAll();
+    }
+
+    /**
+     *
+     */
+    private void destroyIndex(IgniteEx ignite, T3<String, String, String> pair) {
+        IgniteCache<?, ?> cache = ignite.getOrCreateCache(pair.get1());
+
+        String createIdxQryStr = String.format("DROP INDEX %s", pair.get3());
+
+        cache.query(new SqlFieldsQuery(createIdxQryStr)).getAll();
+    }
+
+    /**
+     *
+     */
+    @Target(ElementType.METHOD)
+    @Retention(RetentionPolicy.RUNTIME)
+    protected @interface TestIndex {
+        /**
+         * Index in {@link CacheBlockOnCreateDestoryIndexTest#tests} list.
+         */
+        int value();
+    }
+
+    /**
+     * Index in {@link CacheBlockOnCreateDestoryIndexTest#tests} list.
+     *
+     * @see TestIndex#value()
+     */
+    private CacheBlockOnReadAbstractTest currentTest() {
+        return tests.get(currentTestAnnotation(TestIndex.class).value());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSqlQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSqlQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSqlQueryTest.java
new file mode 100644
index 0000000..f1d96ea
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSqlQueryTest.java
@@ -0,0 +1,131 @@
+/*
+ * 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.cache.distributed;
+
+import java.util.Random;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ *
+ */
+public class CacheBlockOnSqlQueryTest extends CacheBlockOnReadAbstractTest {
+
+    /** {@inheritDoc} */
+    @Override @NotNull protected CacheReadBackgroundOperation<?, ?> getReadOperation() {
+        return new CacheReadBackgroundOperation<Integer, TestingEntity>() {
+            /** Random. */
+            private Random random = new Random();
+
+            /** {@inheritDoc} */
+            @Override protected CacheConfiguration<Integer, TestingEntity> createCacheConfiguration() {
+                return super.createCacheConfiguration().setIndexedTypes(Integer.class, TestingEntity.class);
+            }
+
+            /** {@inheritDoc} */
+            @Override protected Integer createKey(int idx) {
+                return idx;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected TestingEntity createValue(int idx) {
+                return new TestingEntity(idx, idx);
+            }
+
+            /** {@inheritDoc} */
+            @Override public void doRead() {
+                int idx = random.nextInt(entriesCount());
+
+                cache().query(
+                    new SqlQuery<>(TestingEntity.class, "val >= ? and val < ?")
+                        .setArgs(idx, idx + 500)
+                ).getAll();
+            }
+        };
+    }
+
+    /**
+     *
+     */
+    public static class TestingEntity {
+        /** Id. */
+        @QuerySqlField(index = true)
+        public Integer id;
+
+        /** Value. */
+        @QuerySqlField(index = true)
+        public double val;
+
+        /**
+         * Default constructor.
+         */
+        public TestingEntity() {
+        }
+
+        /**
+         * @param id Id.
+         * @param val Value.
+         */
+        public TestingEntity(Integer id, double val) {
+            this.id = id;
+            this.val = val;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStartServerAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9916");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStartServerTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9916");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStopServerAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9916");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStopServerTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9916");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStopBaselineAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9916");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStopBaselineTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9916");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnSqlReadOperationsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnSqlReadOperationsTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnSqlReadOperationsTestSuite.java
new file mode 100644
index 0000000..3be640b
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnSqlReadOperationsTestSuite.java
@@ -0,0 +1,39 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnCreateDestoryIndexTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnSqlQueryTest;
+
+/**
+ * Test suite for cache queries.
+ */
+public class IgniteCacheBlockExchangeOnSqlReadOperationsTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     */
+    public static TestSuite suite() {
+        TestSuite suite = new TestSuite("Do Not Block Read Operations Test Suite");
+
+        suite.addTestSuite(CacheBlockOnSqlQueryTest.class);
+        suite.addTestSuite(CacheBlockOnCreateDestoryIndexTest.class);
+
+        return suite;
+    }
+}


[06/50] [abbrv] ignite git commit: IGNITE-9922 Python thin included in binary release

Posted by sb...@apache.org.
IGNITE-9922 Python thin included in binary release

This closes #5015


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/03cac751
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/03cac751
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/03cac751

Branch: refs/heads/ignite-gg-14206
Commit: 03cac751c36880fe01f3c66f7112738a72e7300b
Parents: 091ace9
Author: Peter Ivanov <mr...@gmail.com>
Authored: Wed Oct 17 19:55:48 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Wed Oct 17 19:55:48 2018 +0300

----------------------------------------------------------------------
 assembly/release-apache-ignite-base.xml | 29 +++++++++++++++++++++++++++-
 1 file changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/03cac751/assembly/release-apache-ignite-base.xml
----------------------------------------------------------------------
diff --git a/assembly/release-apache-ignite-base.xml b/assembly/release-apache-ignite-base.xml
index ca8562c..91baf73 100644
--- a/assembly/release-apache-ignite-base.xml
+++ b/assembly/release-apache-ignite-base.xml
@@ -69,6 +69,22 @@
             <outputDirectory>/platforms/php</outputDirectory>
         </file>
 
+        <!-- Copy Python files. -->
+        <file>
+            <source>modules/platforms/python/LICENSE</source>
+            <outputDirectory>/platforms/python</outputDirectory>
+        </file>
+
+        <file>
+            <source>modules/platforms/python/README.md</source>
+            <outputDirectory>/platforms/python</outputDirectory>
+        </file>
+
+        <file>
+            <source>modules/platforms/python/setup.py</source>
+            <outputDirectory>/platforms/python</outputDirectory>
+        </file>
+
         <!-- Other files. -->
         <file>
             <source>assembly/LICENSE_IGNITE</source>
@@ -225,7 +241,7 @@
             <outputDirectory>/platforms/nodejs/examples</outputDirectory>
         </fileSet>
 
-        <!-- Copy PHP sources.. -->
+        <!-- Copy PHP sources. -->
         <fileSet>
             <directory>modules/platforms/php/src</directory>
             <outputDirectory>/platforms/php/src</outputDirectory>
@@ -236,6 +252,17 @@
             <outputDirectory>/platforms/php/examples</outputDirectory>
         </fileSet>
 
+        <!-- Copy Python sources. -->
+        <fileSet>
+            <directory>modules/platforms/python/pyignite</directory>
+            <outputDirectory>/platforms/python/pyignite</outputDirectory>
+        </fileSet>
+
+        <fileSet>
+            <directory>modules/platforms/python/requirements</directory>
+            <outputDirectory>/platforms/python/requirements</outputDirectory>
+        </fileSet>
+
         <!-- Other files. -->
         <fileSet>
             <directory>bin</directory>


[29/50] [abbrv] ignite git commit: IGNITE-9694 Add tests to check that reading queries are not blocked on exchange events that don't change data visibility - Fixes #4926.

Posted by sb...@apache.org.
IGNITE-9694 Add tests to check that reading queries are not blocked on exchange events that don't change data visibility - Fixes #4926.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3fae41b1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3fae41b1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3fae41b1

Branch: refs/heads/ignite-gg-14206
Commit: 3fae41b1fce89f2f05ff9027cdc37ed84f3a70a0
Parents: 3a4167a
Author: ibessonov <be...@gmail.com>
Authored: Mon Oct 22 14:59:19 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Oct 22 14:59:19 2018 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |   12 +-
 .../ignite/internal/util/lang/GridFunc.java     |    2 +
 .../distributed/CacheBlockOnGetAllTest.java     |  196 +++
 .../CacheBlockOnReadAbstractTest.java           | 1277 ++++++++++++++++++
 .../cache/distributed/CacheBlockOnScanTest.java |   73 +
 .../distributed/CacheBlockOnSingleGetTest.java  |  190 +++
 .../testframework/junits/GridAbstractTest.java  |   33 +
 ...eBlockExchangeOnReadOperationsTestSuite.java |   51 +
 .../testsuites/IgniteCacheTestSuite7.java       |    2 +-
 .../CacheBlockOnCreateDestoryIndexTest.java     |  480 +++++++
 .../distributed/CacheBlockOnSqlQueryTest.java   |  131 ++
 ...ockExchangeOnSqlReadOperationsTestSuite.java |   39 +
 12 files changed, 2475 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 d10a7c7..e6f374a 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
@@ -8169,23 +8169,15 @@ public abstract class IgniteUtils {
             for (Class cls = obj.getClass(); cls != Object.class; cls = cls.getSuperclass()) {
                 for (Field field : cls.getDeclaredFields()) {
                     if (field.getName().equals(fieldName)) {
-                        boolean accessible = field.isAccessible();
-
                         field.setAccessible(true);
 
-                        T val = (T)field.get(obj);
-
-                        if (!accessible)
-                            field.setAccessible(false);
-
-                        return val;
+                        return (T)field.get(obj);
                     }
                 }
             }
         }
         catch (Exception e) {
-            throw new IgniteException("Failed to get field value [fieldName=" + fieldName + ", obj=" + obj + ']',
-                e);
+            throw new IgniteException("Failed to get field value [fieldName=" + fieldName + ", obj=" + obj + ']', e);
         }
 
         throw new IgniteException("Failed to get field value [fieldName=" + fieldName + ", obj=" + obj + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index ce5076b..3b3bbaa 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -94,6 +94,7 @@ import org.apache.ignite.internal.util.lang.gridfunc.TransformFilteringIterator;
 import org.apache.ignite.internal.util.lang.gridfunc.TransformMapView;
 import org.apache.ignite.internal.util.lang.gridfunc.TransformMapView2;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiClosure;
@@ -2172,6 +2173,7 @@ public class GridFunc {
      * @param t2 Second object in pair.
      * @param <T> Type of objects in pair.
      * @return Pair of objects.
+     * @deprecated Use {@link T2} instead.
      */
     @Deprecated
     public static <T> IgnitePair<T> pair(@Nullable T t1, @Nullable T t2) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnGetAllTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnGetAllTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnGetAllTest.java
new file mode 100644
index 0000000..084a431
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnGetAllTest.java
@@ -0,0 +1,196 @@
+/*
+ * 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.cache.distributed;
+
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class CacheBlockOnGetAllTest extends CacheBlockOnReadAbstractTest {
+
+    /** {@inheritDoc} */
+    @Override @NotNull protected CacheReadBackgroundOperation<?, ?> getReadOperation() {
+        return new IntCacheReadBackgroundOperation() {
+            /** Random. */
+            private Random random = new Random();
+
+            /** {@inheritDoc} */
+            @Override public void doRead() {
+                Set<Integer> keys = new HashSet<>();
+
+                for (int i = 0; i < 500; i++)
+                    keys.add(random.nextInt(entriesCount()));
+
+                cache().getAll(keys);
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStopBaselineAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testStopBaselineAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStopBaselineTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testStopBaselineTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testCreateCacheAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testCreateCacheAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testCreateCacheTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testCreateCacheTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testDestroyCacheAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testDestroyCacheAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testDestroyCacheTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testDestroyCacheTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStartServerAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testStartServerAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStartServerTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testStartServerTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStopServerAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testStopServerAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStopServerTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testStopServerTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testUpdateBaselineTopologyAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testUpdateBaselineTopologyAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testUpdateBaselineTopologyTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testUpdateBaselineTopologyTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnReadAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnReadAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnReadAbstractTest.java
new file mode 100644
index 0000000..42b5df0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnReadAbstractTest.java
@@ -0,0 +1,1277 @@
+/*
+ * 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.cache.distributed;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.EventType;
+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.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.events.DiscoveryCustomEvent;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
+import org.apache.ignite.internal.processors.cache.ExchangeActions;
+import org.apache.ignite.internal.processors.cache.ExchangeActions.CacheActionData;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public abstract class CacheBlockOnReadAbstractTest extends GridCommonAbstractTest {
+    /** Default cache entries count. */
+    private static final int DFLT_CACHE_ENTRIES_CNT = 2 * 1024;
+
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** List of baseline nodes started at the beginning of the test. */
+    protected final List<IgniteEx> baseline = new CopyOnWriteArrayList<>();
+
+    /** List of server nodes started at the beginning of the test. */
+    protected final List<IgniteEx> srvs = new CopyOnWriteArrayList<>();
+
+    /** List of client nodes started at the beginning of the test. */
+    protected final List<IgniteEx> clients = new CopyOnWriteArrayList<>();
+
+    /** Start node in client mode. */
+    private volatile boolean startNodesInClientMode;
+
+    /** Latch that is used to wait until all required messages are blocked. */
+    private volatile CountDownLatch cntFinishedReadOperations;
+
+    /** Custom ip finder. */
+    private volatile TcpDiscoveryIpFinder customIpFinder;
+
+    /**
+     * Number of baseline servers to start before test.
+     *
+     * @see Params#baseline()
+     */
+    protected int baselineServersCount() {
+        return currentTestParams().baseline();
+    }
+
+    /**
+     * Number of non-baseline servers to start before test.
+     *
+     * @see Params#servers()
+     */
+    protected int serversCount() {
+        return currentTestParams().servers();
+    }
+
+    /**
+     * Number of clients to start before test.
+     *
+     * @see Params#clients()
+     */
+    protected int clientsCount() {
+        return currentTestParams().clients();
+    }
+
+    /**
+     * Number of backups to configure in caches by default.
+     */
+    protected int backupsCount() {
+        return Math.min(3, baselineServersCount() - 1);
+    }
+
+    /**
+     * Number of milliseconds to warmup reading process. Used to lower fluctuations in run time. Might be 0.
+     *
+     * @see Params#warmup()
+     */
+    protected long warmup() {
+        return currentTestParams().warmup();
+    }
+
+    /**
+     * Number of milliseconds to wait on the potentially blocking operation.
+     *
+     * @see Params#timeout()
+     */
+    protected long timeout() {
+        return currentTestParams().timeout();
+    }
+
+    /**
+     * Cache atomicity mode.
+     *
+     * @see Params#atomicityMode()
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return currentTestParams().atomicityMode();
+    }
+
+    /**
+     * Cache mode.
+     *
+     * @see Params#cacheMode()
+     */
+    protected CacheMode cacheMode() {
+        return currentTestParams().cacheMode();
+    }
+
+    /**
+     * Whether allowing {@link ClusterTopologyCheckedException} as the valid reading result or not.
+     *
+     * @see Params#allowException()
+     */
+    protected boolean allowException() {
+        return currentTestParams().allowException();
+    }
+
+    /**
+     * @param startNodesInClientMode Start nodes on client mode.
+     */
+    public void startNodesInClientMode(boolean startNodesInClientMode) {
+        this.startNodesInClientMode = startNodesInClientMode;
+    }
+
+    /** List of baseline nodes started at the beginning of the test. */
+    public List<? extends IgniteEx> baseline() {
+        return baseline;
+    }
+
+    /** List of server nodes started at the beginning of the test. */
+    public List<? extends IgniteEx> servers() {
+        return srvs;
+    }
+
+    /** List of client nodes started at the beginning of the test. */
+    public List<? extends IgniteEx> clients() {
+        return clients;
+    }
+
+    /**
+     * Annotation to configure test methods in {@link CacheBlockOnReadAbstractTest}. Its values are used throughout
+     * test implementation.
+     */
+    @Target(ElementType.METHOD)
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Params {
+        /**
+         * Number of baseline servers to start before test.
+         */
+        int baseline() default 3;
+
+        /**
+         * Number of non-baseline servers to start before test.
+         */
+        int servers() default 1;
+
+        /**
+         * Number of clients to start before test.
+         */
+        int clients() default 1;
+
+        /**
+         * Number of milliseconds to warmup reading process. Used to lower fluctuations in run time. Might be 0.
+         */
+        long warmup() default 2000L;
+
+        /**
+         * Number of milliseconds to wait on the potentially blocking operation.
+         */
+        long timeout() default 3000L;
+
+        /**
+         * Cache atomicity mode.
+         */
+        CacheAtomicityMode atomicityMode();
+
+        /**
+         * Cache mode.
+         */
+        CacheMode cacheMode();
+
+        /**
+         * Whether allowing {@link ClusterTopologyCheckedException} as the valid reading result or not.
+         */
+        boolean allowException() default false;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(customIpFinder == null ? IP_FINDER : customIpFinder);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration()
+                        .setPersistenceEnabled(true)
+                )
+        );
+
+        cfg.setClientMode(startNodesInClientMode);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        // Checking prerequisites.
+        assertTrue("Positive timeout is required for the test.", timeout() > 0);
+
+        assertTrue("No baseline servers were requested.", baselineServersCount() > 0);
+
+        int idx = 0;
+
+        // Start baseline nodes.
+        for (int i = 0; i < baselineServersCount(); i++)
+            baseline.add(startGrid(idx++));
+
+        // Activate cluster.
+        baseline.get(0).cluster().active(true);
+
+        // Start server nodes in activated cluster.
+        for (int i = 0; i < serversCount(); i++)
+            srvs.add(startGrid(idx++));
+
+        // Start client nodes.
+        startNodesInClientMode(true);
+
+        customIpFinder = new TcpDiscoveryVmIpFinder(false)
+            .setAddresses(
+                Collections.singletonList("127.0.0.1:47500")
+            );
+
+        for (int i = 0; i < clientsCount(); i++)
+            clients.add(startGrid(idx++));
+
+        customIpFinder = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void afterTest() throws Exception {
+        baseline.clear();
+
+        srvs.clear();
+
+        clients.clear();
+
+        grid(0).cluster().active(false);
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testCreateCacheAtomicPartitioned() throws Exception {
+        testCreateCacheTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testCreateCacheAtomicReplicated() throws Exception {
+        testCreateCacheTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testCreateCacheTransactionalPartitioned() throws Exception {
+        testCreateCacheTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testCreateCacheTransactionalReplicated() throws Exception {
+        doTest(
+            asMessagePredicate(CacheBlockOnReadAbstractTest::createCachePredicate),
+            () -> baseline.get(0).createCache(UUID.randomUUID().toString())
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testDestroyCacheAtomicPartitioned() throws Exception {
+        testDestroyCacheTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testDestroyCacheAtomicReplicated() throws Exception {
+        testDestroyCacheTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testDestroyCacheTransactionalPartitioned() throws Exception {
+        testDestroyCacheTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testDestroyCacheTransactionalReplicated() throws Exception {
+        List<String> cacheNames = new ArrayList<>(Arrays.asList(
+            UUID.randomUUID().toString(),
+            UUID.randomUUID().toString(),
+            UUID.randomUUID().toString())
+        );
+
+        for (String cacheName : cacheNames)
+            baseline.get(0).createCache(cacheName);
+
+        doTest(
+            asMessagePredicate(CacheBlockOnReadAbstractTest::destroyCachePredicate),
+            () -> baseline.get(0).destroyCache(cacheNames.remove(0))
+        );
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void _testStartClient() throws Exception {
+        startNodesInClientMode(true);
+
+        doTest(
+            asMessagePredicate(discoEvt -> discoEvt.type() == EventType.EVT_NODE_JOINED),
+            () -> {
+                for (int i = 0; i < baselineServersCount() - 2; i++)
+                    cntFinishedReadOperations.countDown();
+
+                customIpFinder = new TcpDiscoveryVmIpFinder(false)
+                    .setAddresses(
+                        Collections.singletonList("127.0.0.1:47500")
+                    );
+
+                startGrid(UUID.randomUUID().toString());
+
+                customIpFinder = null;
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void _testStopClient() throws Exception {
+        customIpFinder = new TcpDiscoveryVmIpFinder(false)
+            .setAddresses(
+                Collections.singletonList("127.0.0.1:47500")
+            );
+
+        startNodesInClientMode(true);
+
+        for (int i = 0; i < 3; i++)
+            clients.add(startGrid(UUID.randomUUID().toString()));
+
+        customIpFinder = null;
+
+        doTest(
+            asMessagePredicate(discoEvt -> discoEvt.type() == EventType.EVT_NODE_LEFT),
+            () -> {
+                for (int i = 0; i < baselineServersCount() - 2; i++)
+                    cntFinishedReadOperations.countDown();
+
+                stopGrid(clients.remove(clients.size() - 1).name());
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testStartServerAtomicPartitioned() throws Exception {
+        testStartServerTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testStartServerAtomicReplicated() throws Exception {
+        testStartServerTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testStartServerTransactionalPartitioned() throws Exception {
+        testStartServerTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testStartServerTransactionalReplicated() throws Exception {
+        startNodesInClientMode(false);
+
+        doTest(
+            asMessagePredicate(discoEvt -> discoEvt.type() == EventType.EVT_NODE_JOINED),
+            () -> startGrid(UUID.randomUUID().toString())
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(servers = 4, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testStopServerAtomicPartitioned() throws Exception {
+        testStopServerTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(servers = 4, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testStopServerAtomicReplicated() throws Exception {
+        testStopServerTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(servers = 4, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testStopServerTransactionalPartitioned() throws Exception {
+        testStopServerTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(servers = 4, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testStopServerTransactionalReplicated() throws Exception {
+        doTest(
+            asMessagePredicate(discoEvt -> discoEvt.type() == EventType.EVT_NODE_LEFT),
+            () -> stopGrid(srvs.remove(srvs.size() - 1).name())
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 4, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testRestartBaselineAtomicPartitioned() throws Exception {
+        testRestartBaselineTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 4, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testRestartBaselineAtomicReplicated() throws Exception {
+        testRestartBaselineTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 4, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testRestartBaselineTransactionalPartitioned() throws Exception {
+        testRestartBaselineTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 4, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testRestartBaselineTransactionalReplicated() throws Exception {
+        doTest(
+            asMessagePredicate(discoEvt -> discoEvt.type() == EventType.EVT_NODE_JOINED),
+            () -> {
+                IgniteEx node = baseline.get(baseline.size() - 1);
+
+                TestRecordingCommunicationSpi.spi(node).stopBlock();
+
+                stopGrid(node.name());
+
+                for (int i = 0; i < baselineServersCount() - 2; i++)
+                    cntFinishedReadOperations.countDown();
+
+                startGrid(node.name());
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testUpdateBaselineTopologyAtomicPartitioned() throws Exception {
+        testUpdateBaselineTopologyTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testUpdateBaselineTopologyAtomicReplicated() throws Exception {
+        testUpdateBaselineTopologyTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testUpdateBaselineTopologyTransactionalPartitioned() throws Exception {
+        testUpdateBaselineTopologyTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(timeout = 5000L, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testUpdateBaselineTopologyTransactionalReplicated() throws Exception {
+        doTest(
+            asMessagePredicate(discoEvt -> {
+                if (discoEvt instanceof DiscoveryCustomEvent) {
+                    DiscoveryCustomEvent discoCustomEvt = (DiscoveryCustomEvent)discoEvt;
+
+                    DiscoveryCustomMessage customMsg = discoCustomEvt.customMessage();
+
+                    return customMsg instanceof ChangeGlobalStateMessage;
+                }
+
+                return false;
+            }),
+            () -> {
+                startNodesInClientMode(false);
+
+                IgniteEx ignite = startGrid(UUID.randomUUID().toString());
+
+                baseline.get(0).cluster().setBaselineTopology(baseline.get(0).context().discovery().topologyVersion());
+
+                baseline.add(ignite);
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 9, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    public void testStopBaselineAtomicPartitioned() throws Exception {
+        testStopBaselineTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 9, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    public void testStopBaselineAtomicReplicated() throws Exception {
+        testStopBaselineTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 9, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    public void testStopBaselineTransactionalPartitioned() throws Exception {
+        testStopBaselineTransactionalReplicated();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Params(baseline = 9, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    public void testStopBaselineTransactionalReplicated() throws Exception {
+        AtomicInteger cntDownCntr = new AtomicInteger(0);
+
+        doTest(
+            asMessagePredicate(discoEvt -> discoEvt.type() == EventType.EVT_NODE_LEFT),
+            () -> {
+                IgniteEx node = baseline.get(baseline.size() - cntDownCntr.get() - 1);
+
+                TestRecordingCommunicationSpi.spi(node).stopBlock();
+
+                cntDownCntr.incrementAndGet();
+
+                for (int i = 0; i < cntDownCntr.get(); i++)
+                    cntFinishedReadOperations.countDown(); // This node and previously stopped nodes as well.
+
+                stopGrid(node.name());
+            }
+        );
+    }
+
+    /**
+     * Checks that given discovery event is from "Create cache" operation.
+     *
+     * @param discoEvt Discovery event.
+     */
+    private static boolean createCachePredicate(DiscoveryEvent discoEvt) {
+        if (discoEvt instanceof DiscoveryCustomEvent) {
+
+            DiscoveryCustomEvent discoCustomEvt = (DiscoveryCustomEvent)discoEvt;
+
+            DiscoveryCustomMessage customMsg = discoCustomEvt.customMessage();
+
+            if (customMsg instanceof DynamicCacheChangeBatch) {
+                DynamicCacheChangeBatch cacheChangeBatch = (DynamicCacheChangeBatch)customMsg;
+
+                ExchangeActions exchangeActions = U.field(cacheChangeBatch, "exchangeActions");
+
+                Collection<CacheActionData> startRequests = exchangeActions.cacheStartRequests();
+
+                return !startRequests.isEmpty();
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks that given discovery event is from "Destroy cache" operation.
+     *
+     * @param discoEvt Discovery event.
+     */
+    private static boolean destroyCachePredicate(DiscoveryEvent discoEvt) {
+        if (discoEvt instanceof DiscoveryCustomEvent) {
+
+            DiscoveryCustomEvent discoCustomEvt = (DiscoveryCustomEvent)discoEvt;
+
+            DiscoveryCustomMessage customMsg = discoCustomEvt.customMessage();
+
+            if (customMsg instanceof DynamicCacheChangeBatch) {
+                DynamicCacheChangeBatch cacheChangeBatch = (DynamicCacheChangeBatch)customMsg;
+
+                ExchangeActions exchangeActions = U.field(cacheChangeBatch, "exchangeActions");
+
+                Collection<CacheActionData> stopRequests = exchangeActions.cacheStopRequests();
+
+                return !stopRequests.isEmpty();
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Read operation tat is going to be executed during blocking operation.
+     */
+    @NotNull protected abstract CacheReadBackgroundOperation getReadOperation();
+
+    /**
+     * Checks that {@code block} closure doesn't block read operation.
+     * Does it for client, baseline and regular server node.
+     *
+     * @param blockMsgPred Predicate that check whether the message corresponds to the {@code block} or not.
+     * @param block Blocking operation.
+     * @throws Exception If failed.
+     */
+    public void doTest(Predicate<Message> blockMsgPred, RunnableX block) throws Exception {
+        BackgroundOperation backgroundOperation = new BlockMessageOnBaselineBackgroundOperation(
+            block,
+            blockMsgPred
+        );
+
+        CacheReadBackgroundOperation<?, ?> readOperation = getReadOperation();
+
+        readOperation.initCache(baseline.get(0), true);
+
+        // Warmup.
+        if (warmup() > 0) {
+            try (AutoCloseable read = readOperation.start()) {
+                Thread.sleep(warmup());
+            }
+
+            assertEquals(
+                readOperation.readOperationsFailed() + " read operations failed during warmup.",
+                0,
+                readOperation.readOperationsFailed()
+            );
+
+            assertTrue(
+                "No read operations were finished during warmup.",
+                readOperation.readOperationsFinishedUnderBlock() > 0
+            );
+        }
+
+
+        doTest0(clients.get(0), readOperation, backgroundOperation);
+
+        doTest0(srvs.get(0), readOperation, backgroundOperation);
+
+        doTest0(baseline.get(0), readOperation, backgroundOperation);
+
+
+        try (AutoCloseable read = readOperation.start()) {
+            Thread.sleep(500L);
+        }
+
+        assertEquals(
+            readOperation.readOperationsFailed() + " read operations failed during finish stage.",
+            0,
+            readOperation.readOperationsFailed()
+        );
+
+        assertTrue(
+            "No read operations were finished during finish stage.",
+            readOperation.readOperationsFinishedUnderBlock() > 0
+        );
+    }
+
+    /**
+     * Internal part for {@link CacheBlockOnReadAbstractTest#doTest(Predicate, RunnableX)}.
+     *
+     * @param ignite Ignite instance. Client / baseline / server node.
+     * @param readOperation Read operation.
+     * @param backgroundOperation Background operation.
+     */
+    private void doTest0(
+        IgniteEx ignite,
+        CacheReadBackgroundOperation<?, ?> readOperation,
+        BackgroundOperation backgroundOperation
+    ) throws Exception {
+        // Reinit internal cache state with given ignite instance.
+        readOperation.initCache(ignite, false);
+
+        cntFinishedReadOperations = new CountDownLatch(baseline.size() - 1);
+
+        // Read while potentially blocking operation is executing.
+        try (AutoCloseable block = backgroundOperation.start()) {
+            cntFinishedReadOperations.await(5 * timeout(), TimeUnit.MILLISECONDS);
+
+            // Possible if test itself is wrong.
+            assertEquals("Messages weren't blocked in time", 0, cntFinishedReadOperations.getCount());
+
+            try (AutoCloseable read = readOperation.start()) {
+                Thread.sleep(timeout());
+            }
+        }
+        finally {
+            cntFinishedReadOperations = null;
+        }
+
+        log.info("Operations finished: " + readOperation.readOperationsFinishedUnderBlock());
+        log.info("Longest operation took " + readOperation.maxReadDuration() + "ms");
+
+        // None of read operations should fail.
+        assertEquals(
+            readOperation.readOperationsFailed() + " read operations failed.",
+            0,
+            readOperation.readOperationsFailed()
+        );
+
+        assertTrue(
+            "No read operations were finished during timeout.",
+            readOperation.readOperationsFinishedUnderBlock() > 0
+        );
+
+        // There were no operations as long as blocking timeout.
+        assertNotAlmostEqual(timeout(), readOperation.maxReadDuration());
+
+        // On average every read operation was much faster then blocking timeout.
+        double avgDuration = (double)timeout() / readOperation.readOperationsFinishedUnderBlock();
+
+        assertTrue("Avarage duration was too long.",avgDuration < timeout() * 0.25);
+    }
+
+    /**
+     * Utility class that allows to start and stop some background operation many times.
+     */
+    protected abstract static class BackgroundOperation {
+        /** */
+        private IgniteInternalFuture<?> fut;
+
+        /**
+         * Invoked strictly before background thread is started.
+         */
+        protected void init() {
+            // No-op.
+        }
+
+        /**
+         * Operation itself. Will be executed in separate thread. Thread interruption has to be considered as a valid
+         * way to stop operation.
+         */
+        protected abstract void execute();
+
+        /**
+         * @return Allowed time to wait in {@link BackgroundOperation#stop()} method before canceling background thread.
+         */
+        protected abstract long stopTimeout();
+
+        /**
+         * Start separate thread and execute method {@link BackgroundOperation#execute()} in it.
+         *
+         * @return {@link AutoCloseable} that invokes {@link BackgroundOperation#stop()} on closing.
+         */
+        AutoCloseable start() {
+            if (fut != null)
+                throw new UnsupportedOperationException("Only one simultanious operation is allowed");
+
+            init();
+
+            CountDownLatch threadStarted = new CountDownLatch(1);
+
+            fut = GridTestUtils.runAsync(() -> {
+                try {
+                    threadStarted.countDown();
+
+                    execute();
+                }
+                catch (Exception e) {
+                    throw new IgniteException("Unexpected exception in background operation thread", e);
+                }
+            });
+
+            try {
+                threadStarted.await();
+            }
+            catch (InterruptedException e) {
+                try {
+                    fut.cancel();
+                }
+                catch (IgniteCheckedException e1) {
+                    e.addSuppressed(e1);
+                }
+
+                throw new IgniteException(e);
+            }
+
+            return this::stop;
+        }
+
+        /**
+         * Interrupt the operation started in {@link BackgroundOperation#start()} method and join interrupted thread.
+         */
+        void stop() throws Exception {
+            if (fut == null)
+                return;
+
+            try {
+                fut.get(stopTimeout());
+            }
+            catch (IgniteFutureTimeoutCheckedException e) {
+                fut.cancel();
+
+                fut.get();
+            }
+            finally {
+                fut = null;
+            }
+        }
+    }
+
+    /**
+     * @param discoEvtPred Predicate that tests discovery events.
+     * @return New predicate that test any message based on {@code discoEvtPred} predicate.
+     */
+    public static Predicate<Message> asMessagePredicate(Predicate<DiscoveryEvent> discoEvtPred) {
+        return msg -> {
+            if (msg instanceof GridDhtPartitionsFullMessage) {
+                GridDhtPartitionsFullMessage fullMsg = (GridDhtPartitionsFullMessage)msg;
+
+                GridDhtPartitionExchangeId exchangeId = fullMsg.exchangeId();
+
+                if (exchangeId != null)
+                    return discoEvtPred.test(U.field(exchangeId, "discoEvt"));
+            }
+
+            return false;
+        };
+    }
+
+    /**
+     * Background operation that executes some node request and doesn't allow its messages to be fully processed until
+     * operation is stopped.
+     */
+    protected class BlockMessageOnBaselineBackgroundOperation extends BackgroundOperation {
+        /** */
+        private final RunnableX block;
+
+        /** */
+        private final Predicate<Message> blockMsg;
+
+        /**
+         * @param block Blocking operation.
+         * @param blockMsgPred Predicate that checks whether to block message or not.
+         *
+         * @see BlockMessageOnBaselineBackgroundOperation#blockMessage(ClusterNode, Message)
+         */
+        protected BlockMessageOnBaselineBackgroundOperation(
+            RunnableX block,
+            Predicate<Message> blockMsgPred
+        ) {
+            this.block = block;
+            blockMsg = blockMsgPred;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void execute() {
+            for (IgniteEx server : baseline) {
+                TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(server);
+
+                spi.blockMessages(this::blockMessage);
+            }
+
+            block.run();
+        }
+
+        /**
+         * Function to pass into {@link TestRecordingCommunicationSpi#blockMessages(IgniteBiPredicate)}.
+         *
+         * @param node Node that receives message.
+         * @param msg Message.
+         * @return Whether the given message should be blocked or not.
+         */
+        private boolean blockMessage(ClusterNode node, Message msg) {
+            boolean block = blockMsg.test(msg)
+                && baseline.stream().map(IgniteEx::name).anyMatch(node.consistentId()::equals);
+
+            if (block)
+                cntFinishedReadOperations.countDown();
+
+            return block;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected long stopTimeout() {
+            // Should be big enough so thread will stop by it's own. Otherwise test will fail, but that's fine.
+            return 30_000L;
+        }
+
+        /** {@inheritDoc} */
+        @Override void stop() throws Exception {
+            for (IgniteEx server : baseline) {
+                TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(server);
+
+                spi.stopBlock();
+            }
+
+            super.stop();
+        }
+    }
+
+
+    /**
+     * Runnable that can throw exceptions.
+     */
+    @FunctionalInterface
+    public interface RunnableX extends Runnable {
+        /**
+         * Closure body.
+         *
+         * @throws Exception If failed.
+         */
+        void runx() throws Exception;
+
+        /** {@inheritdoc} */
+        @Override default void run() {
+            try {
+                runx();
+            }
+            catch (Exception e) {
+                throw new IgniteException(e);
+            }
+        }
+    }
+
+    /**
+     * {@link BackgroundOperation} implementation for cache reading operations.
+     */
+    protected abstract class ReadBackgroundOperation extends BackgroundOperation {
+
+        /** Counter for successfully finished operations. */
+        private final AtomicInteger readOperationsFinishedUnderBlock = new AtomicInteger();
+
+        /** Counter for failed operations. */
+        private final AtomicInteger readOperationsFailed = new AtomicInteger();
+
+        /** Duration of the longest read operation. */
+        private final AtomicLong maxReadDuration = new AtomicLong(-1);
+
+        /**
+         * Do single iteration of reading operation. Will be executed in a loop.
+         */
+        protected abstract void doRead() throws Exception;
+
+
+        /** {@inheritDoc} */
+        @Override protected void init() {
+            readOperationsFinishedUnderBlock.set(0);
+
+            readOperationsFailed.set(0);
+
+            maxReadDuration.set(-1);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void execute() {
+            Set<String> loggedMessages = new HashSet<>();
+
+            while (!Thread.currentThread().isInterrupted()) {
+                long prevTs = System.currentTimeMillis();
+
+                try {
+                    doRead();
+
+                    readOperationsFinishedUnderBlock.incrementAndGet();
+                }
+                catch (Exception e) {
+                    boolean threadInterrupted = X.hasCause(e,
+                        InterruptedException.class,
+                        IgniteInterruptedException.class,
+                        IgniteInterruptedCheckedException.class
+                    );
+
+                    if (threadInterrupted)
+                        Thread.currentThread().interrupt();
+                    else if (allowException() && X.hasCause(e, ClusterTopologyCheckedException.class))
+                        readOperationsFinishedUnderBlock.incrementAndGet();
+                    else {
+                        readOperationsFailed.incrementAndGet();
+
+                        if (loggedMessages.add(e.getMessage()))
+                            log.error("Error during read operation execution", e);
+
+                        continue;
+                    }
+                }
+
+                maxReadDuration.set(Math.max(maxReadDuration.get(), System.currentTimeMillis() - prevTs));
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected long stopTimeout() {
+            return 0;
+        }
+
+        /**
+         * @return Number of successfully finished operations.
+         */
+        public int readOperationsFinishedUnderBlock() {
+            return readOperationsFinishedUnderBlock.get();
+        }
+
+        /**
+         * @return Number of failed operations.
+         */
+        public int readOperationsFailed() {
+            return readOperationsFailed.get();
+        }
+
+        /**
+         * @return Duration of the longest read operation.
+         */
+        public long maxReadDuration() {
+            return maxReadDuration.get();
+        }
+    }
+
+    /**
+     *
+     */
+    protected abstract class CacheReadBackgroundOperation<KeyType, ValueType> extends ReadBackgroundOperation {
+        /**
+         * {@link CacheReadBackgroundOperation#cache()} method backing field. Updated on each
+         * {@link CacheReadBackgroundOperation#initCache(IgniteEx, boolean)} invocation.
+         */
+        private IgniteCache<KeyType, ValueType> cache;
+
+        /**
+         * Reinit internal cache using passed ignite instance and fill it with data if required.
+         *
+         * @param ignite Node to get or create cache from.
+         * @param fillData Whether the cache should be filled with new data or not.
+         */
+        public void initCache(IgniteEx ignite, boolean fillData) {
+            cache = ignite.getOrCreateCache(
+                createCacheConfiguration()
+                    .setAtomicityMode(atomicityMode())
+                    .setCacheMode(cacheMode())
+            );
+
+            if (fillData) {
+                try (IgniteDataStreamer<KeyType, ValueType> dataStreamer = ignite.dataStreamer(cache.getName())) {
+                    dataStreamer.allowOverwrite(true);
+
+                    for (int i = 0; i < entriesCount(); i++)
+                        dataStreamer.addData(createKey(i), createValue(i));
+                }
+            }
+        }
+
+        /**
+         * @return Cache configuration.
+         */
+        protected CacheConfiguration<KeyType, ValueType> createCacheConfiguration() {
+            return new CacheConfiguration<KeyType, ValueType>(DEFAULT_CACHE_NAME)
+                .setBackups(backupsCount())
+                .setAffinity(
+                    new RendezvousAffinityFunction()
+                        .setPartitions(32)
+                );
+        }
+
+        /**
+         * @return Current cache.
+         */
+        protected final IgniteCache<KeyType, ValueType> cache() {
+            return cache;
+        }
+
+        /**
+         * @return Count of cache entries to create in {@link CacheReadBackgroundOperation#initCache(IgniteEx, boolean)}
+         * method.
+         */
+        protected int entriesCount() {
+            return DFLT_CACHE_ENTRIES_CNT;
+        }
+
+        /**
+         * @param idx Unique number.
+         * @return Key to be used for inserting into cache.
+         * @see CacheReadBackgroundOperation#createValue(int)
+         */
+        protected abstract KeyType createKey(int idx);
+
+        /**
+         * @param idx Unique number.
+         * @return Value to be used for inserting into cache.
+         * @see CacheReadBackgroundOperation#createKey(int)
+         */
+        protected abstract ValueType createValue(int idx);
+    }
+
+    /**
+     * {@link CacheReadBackgroundOperation} implementation for (int -> int) cache. Keys and values are equal by default.
+     */
+    protected abstract class IntCacheReadBackgroundOperation
+        extends CacheReadBackgroundOperation<Integer, Integer> {
+        /** {@inheritDoc} */
+        @Override protected Integer createKey(int idx) {
+            return idx;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Integer createValue(int idx) {
+            return idx;
+        }
+    }
+
+    /**
+     * @return {@link Params} annotation object from the current test method.
+     */
+    protected Params currentTestParams() {
+        Params params = currentTestAnnotation(Params.class);
+
+        assertNotNull("Test " + getName() + " is not annotated with @Param annotation.", params);
+
+        return params;
+    }
+
+    /**
+     * Assert that two numbers are close to each other.
+     */
+    private static void assertAlmostEqual(long exp, long actual) {
+        assertTrue(String.format("Numbers differ too much [exp=%d, actual=%d]", exp, actual), almostEqual(exp, actual));
+    }
+
+    /**
+     * Assert that two numbers are not close to each other.
+     */
+    private static void assertNotAlmostEqual(long exp, long actual) {
+        assertFalse(String.format("Numbers are almost equal [exp=%d, actual=%d]", exp, actual), almostEqual(exp, actual));
+    }
+
+    /**
+     * Check that two numbers are close to each other.
+     */
+    private static boolean almostEqual(long exp, long actual) {
+        double rel = (double)(actual - exp) / exp;
+
+        return Math.abs(rel) < 0.05;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnScanTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnScanTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnScanTest.java
new file mode 100644
index 0000000..2912d05
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnScanTest.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.cache.distributed;
+
+import java.util.Objects;
+import java.util.Random;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class CacheBlockOnScanTest extends CacheBlockOnReadAbstractTest {
+
+    /** {@inheritDoc} */
+    @Override @NotNull protected CacheReadBackgroundOperation<?, ?> getReadOperation() {
+        return new IntCacheReadBackgroundOperation() {
+            /** Random. */
+            private Random random = new Random();
+
+            /** {@inheritDoc} */
+            @Override public void doRead() {
+                int idx = random.nextInt(entriesCount());
+
+                cache().query(new ScanQuery<>((k, v) -> Objects.equals(k, idx))).getAll();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 9, atomicityMode = ATOMIC, cacheMode = PARTITIONED, allowException = true)
+    @Override public void testStopBaselineAtomicPartitioned() throws Exception {
+        super.testStopBaselineAtomicPartitioned();
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 9, atomicityMode = ATOMIC, cacheMode = REPLICATED, allowException = true)
+    @Override public void testStopBaselineAtomicReplicated() throws Exception {
+        super.testStopBaselineAtomicReplicated();
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 9, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED, allowException = true)
+    @Override public void testStopBaselineTransactionalPartitioned() throws Exception {
+        super.testStopBaselineTransactionalPartitioned();
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 9, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED, allowException = true)
+    @Override public void testStopBaselineTransactionalReplicated() throws Exception {
+        super.testStopBaselineTransactionalReplicated();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSingleGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSingleGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSingleGetTest.java
new file mode 100644
index 0000000..fc181be
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBlockOnSingleGetTest.java
@@ -0,0 +1,190 @@
+/*
+ * 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.cache.distributed;
+
+import java.util.Random;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class CacheBlockOnSingleGetTest extends CacheBlockOnReadAbstractTest {
+
+    /** {@inheritDoc} */
+    @Override @NotNull protected CacheReadBackgroundOperation<?, ?> getReadOperation() {
+        return new IntCacheReadBackgroundOperation() {
+            /** Random. */
+            private Random random = new Random();
+
+            /** {@inheritDoc} */
+            @Override public void doRead() {
+                for (int i = 0; i < 300; i++)
+                    cache().get(random.nextInt(entriesCount()));
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStopBaselineAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testStopBaselineAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStopBaselineTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testStopBaselineTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9915");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testCreateCacheAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testCreateCacheAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testCreateCacheTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testCreateCacheTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testDestroyCacheAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testDestroyCacheAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testDestroyCacheTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testDestroyCacheTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStartServerAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testStartServerAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStartServerTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testStartServerTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testStopServerAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testStopServerAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testStopServerTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testStopServerTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = PARTITIONED)
+    @Override public void testUpdateBaselineTopologyAtomicPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = ATOMIC, cacheMode = REPLICATED)
+    @Override public void testUpdateBaselineTopologyAtomicReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = PARTITIONED)
+    @Override public void testUpdateBaselineTopologyTransactionalPartitioned() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+
+    /** {@inheritDoc} */
+    @Params(baseline = 1, atomicityMode = TRANSACTIONAL, cacheMode = REPLICATED)
+    @Override public void testUpdateBaselineTopologyTransactionalReplicated() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9883");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
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 f1d6682..057087e 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
@@ -19,6 +19,7 @@ package org.apache.ignite.testframework.junits;
 
 import java.io.ObjectStreamException;
 import java.io.Serializable;
+import java.lang.annotation.Annotation;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
@@ -118,6 +119,7 @@ import org.apache.log4j.Logger;
 import org.apache.log4j.PatternLayout;
 import org.apache.log4j.Priority;
 import org.apache.log4j.RollingFileAppender;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.springframework.beans.BeansException;
 import org.springframework.context.ApplicationContext;
@@ -197,6 +199,9 @@ public abstract class GridAbstractTest extends TestCase {
     /** Number of tests. */
     private int testCnt;
 
+    /** Lazily initialized current test method. */
+    private volatile Method currTestMtd;
+
     /**
      *
      */
@@ -670,6 +675,34 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @return Current test method.
+     * @throws NoSuchMethodError If method wasn't found for some reason.
+     */
+    @NotNull protected Method currentTestMethod() {
+        if (currTestMtd == null) {
+            try {
+                currTestMtd = getClass().getMethod(getName());
+            }
+            catch (NoSuchMethodException e) {
+                throw new NoSuchMethodError("Current test method is not found: " + getName());
+            }
+        }
+
+        return currTestMtd;
+    }
+
+    /**
+     * Search for the annotation of the given type in current test method.
+     *
+     * @param annotationCls Type of annotation to look for.
+     * @param <A> Annotation type.
+     * @return Instance of annotation if it is present in test method.
+     */
+    @Nullable protected <A extends Annotation> A currentTestAnnotation(Class<A> annotationCls) {
+        return currentTestMethod().getAnnotation(annotationCls);
+    }
+
+    /**
      * @return Started grid.
      * @throws Exception If anything failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnReadOperationsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnReadOperationsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnReadOperationsTestSuite.java
new file mode 100755
index 0000000..79ec18b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBlockExchangeOnReadOperationsTestSuite.java
@@ -0,0 +1,51 @@
+/*
+ * 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.testsuites;
+
+import java.util.Set;
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnGetAllTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnScanTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnSingleGetTest;
+
+/**
+ * Test suite.
+ */
+public class IgniteCacheBlockExchangeOnReadOperationsTestSuite extends TestSuite {
+    /**
+     * @return IgniteCache test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        return suite(null);
+    }
+
+    /**
+     * @param ignoredTests Tests to ignore.
+     * @return Test suite.
+     */
+    public static TestSuite suite(Set<Class> ignoredTests) {
+        TestSuite suite = new TestSuite("Do Not Block Read Operations Test Suite");
+
+        suite.addTestSuite(CacheBlockOnSingleGetTest.class);
+        suite.addTestSuite(CacheBlockOnGetAllTest.class);
+        suite.addTestSuite(CacheBlockOnScanTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3fae41b1/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
index 6c48ecc..d0734a8 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
@@ -33,9 +33,9 @@ import org.apache.ignite.internal.processors.cache.WalModeChangeAdvancedSelfTest
 import org.apache.ignite.internal.processors.cache.WalModeChangeCoordinatorNotAffinityNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.WalModeChangeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartitionsTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheDataLossOnPartitionMoveTest;
 import org.apache.ignite.internal.processors.cache.distributed.CachePageWriteLockUnlockTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheRentingStateRepairTest;
-import org.apache.ignite.internal.processors.cache.distributed.CacheDataLossOnPartitionMoveTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheStartWithLoadTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingPartitionCountersTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingWithAsyncClearingTest;


[46/50] [abbrv] ignite git commit: IGNITE-9663: MVCC: Fixes to transaction mapping. This closes #5018.

Posted by sb...@apache.org.
IGNITE-9663: MVCC: Fixes to transaction mapping. This closes #5018.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d76ff542
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d76ff542
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d76ff542

Branch: refs/heads/ignite-gg-14206
Commit: d76ff54264498f49f070d0ff5e6d7d67f29d93ba
Parents: 685c008
Author: rkondakov <ko...@mail.ru>
Authored: Tue Oct 23 14:59:12 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 23 14:59:12 2018 +0300

----------------------------------------------------------------------
 .../near/GridNearTxEnlistFuture.java            |  4 +-
 .../near/GridNearTxQueryEnlistFuture.java       |  3 +-
 .../GridNearTxQueryResultsEnlistFuture.java     |  4 +-
 .../cache/mvcc/MvccProcessorImpl.java           |  9 ++-
 .../cache/mvcc/MvccQueryTrackerImpl.java        | 16 ++++-
 .../cache/mvcc/msg/MvccSnapshotResponse.java    |  4 ++
 .../cache/mvcc/CacheMvccAbstractTest.java       | 67 +++++++++++++++++++-
 .../query/h2/database/H2TreeFilterClosure.java  | 19 +++++-
 .../query/h2/database/H2TreeIndex.java          |  2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  5 +-
 ...ccPartitionedSqlCoordinatorFailoverTest.java |  9 +++
 11 files changed, 121 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
index ed75c36..3ddedd0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
@@ -575,9 +575,7 @@ public class GridNearTxEnlistFuture extends GridNearTxAbstractEnlistFuture<GridC
         if (err == null && res.error() != null)
             err = res.error();
 
-        if (X.hasCause(err, ClusterTopologyCheckedException.class))
-            tx.removeMapping(nodeId);
-        else if (res != null)
+        if (res != null)
             tx.mappings().get(nodeId).addBackups(res.newDhtNodes());
 
         if (err != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
index 8528345..70fbd1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
@@ -363,8 +363,7 @@ public class GridNearTxQueryEnlistFuture extends GridNearTxQueryAbstractEnlistFu
                 completed = true;
             }
 
-            if (X.hasCause(err, ClusterTopologyCheckedException.class)
-                || (res != null && res.removeMapping())) {
+            if (res != null && res.removeMapping()) {
                 GridDistributedTxMapping m = tx.mappings().get(node.id());
 
                 assert m != null && m.empty();

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
index f375405..9783582 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
@@ -552,9 +552,7 @@ public class GridNearTxQueryResultsEnlistFuture extends GridNearTxQueryAbstractE
         if (err == null && res.error() != null)
             err = res.error();
 
-        if (X.hasCause(err, ClusterTopologyCheckedException.class))
-            tx.removeMapping(nodeId);
-        else if (res != null)
+        if (res != null)
             tx.mappings().get(nodeId).addBackups(res.newDhtNodes());
 
         if (err != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index e58151f..a304bef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -1244,10 +1244,13 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                                                 && crd.coordinatorVersion() >= snapshot.coordinatorVersion();
 
                                             for (TxKey key : waitMap.keySet()) {
-                                                assert key.major() == snapshot.coordinatorVersion()
+                                                if (!( key.major() == snapshot.coordinatorVersion()
                                                     && key.minor() > snapshot.cleanupVersion()
-                                                    || key.major() > snapshot.coordinatorVersion() :
-                                                    "key=" + key + ", snapshot=" + snapshot;
+                                                    || key.major() > snapshot.coordinatorVersion())) {
+                                                    byte state = state(key.major(), key.minor());
+
+                                                    assert state == TxState.ABORTED : "tx state=" + state;
+                                                }
                                             }
                                         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java
index 9a767ec..7296d66 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java
@@ -61,6 +61,9 @@ public class MvccQueryTrackerImpl implements MvccQueryTracker {
     /** */
     private final boolean canRemap;
 
+    /** */
+    private boolean done;
+
     /**
      * @param cctx Cache context.
      */
@@ -136,6 +139,9 @@ public class MvccQueryTrackerImpl implements MvccQueryTracker {
 
     /** {@inheritDoc} */
     @Override public void onDone() {
+        if (!checkDone())
+            return;
+
         MvccProcessor prc = cctx.shared().coordinators();
 
         MvccSnapshot snapshot = snapshot();
@@ -151,7 +157,7 @@ public class MvccQueryTrackerImpl implements MvccQueryTracker {
     @Override public IgniteInternalFuture<Void> onDone(@NotNull GridNearTxLocal tx, boolean commit) {
         MvccSnapshot snapshot = snapshot(), txSnapshot = tx.mvccSnapshot();
 
-        if (snapshot == null && txSnapshot == null)
+        if (!checkDone() || snapshot == null && txSnapshot == null)
             return commit ? new GridFinishedFuture<>() : null;
 
         MvccProcessor prc = cctx.shared().coordinators();
@@ -328,6 +334,14 @@ public class MvccQueryTrackerImpl implements MvccQueryTracker {
         return true;
     }
 
+    /** */
+    private synchronized boolean checkDone() {
+        if (!done)
+            return done = true;
+
+        return false;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(MvccQueryTrackerImpl.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
index 2c22616..c5a8494 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java
@@ -24,6 +24,8 @@ import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccLongList;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotWithoutTxs;
+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.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -49,9 +51,11 @@ public class MvccSnapshotResponse implements MvccMessage, MvccSnapshot, MvccLong
 
     /** */
     @GridDirectTransient
+    @GridToStringExclude
     private int txsCnt;
 
     /** */
+    @GridToStringInclude
     private long[] txs;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
index 57f714a..5b32347 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
@@ -40,6 +40,7 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 import javax.cache.Cache;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -63,12 +64,17 @@ import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
+import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.lang.GridInClosure3;
@@ -89,6 +95,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionException;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 
@@ -1475,8 +1482,64 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
      * @param e Exception.
      */
     protected void handleTxException(Exception e) {
-        if (log.isTraceEnabled())
-            log.trace("Exception during tx execution: " + X.getFullStackTrace(e));
+        if (log.isDebugEnabled())
+            log.debug("Exception during tx execution: " + X.getFullStackTrace(e));
+
+        if (X.hasCause(e, IgniteFutureCancelledCheckedException.class))
+            return;
+
+        if (X.hasCause(e, ClusterTopologyException.class))
+            return;
+
+        if (X.hasCause(e, ClusterTopologyCheckedException.class))
+            return;
+
+        if (X.hasCause(e, IgniteTxRollbackCheckedException.class))
+            return;
+
+        if (X.hasCause(e, TransactionException.class))
+            return;
+
+        if (X.hasCause(e, IgniteTxTimeoutCheckedException.class))
+            return;
+
+        if (X.hasCause(e, CacheException.class)) {
+            CacheException cacheEx = X.cause(e, CacheException.class);
+
+            if (cacheEx != null && cacheEx.getMessage() != null) {
+                if (cacheEx.getMessage().contains("Data node has left the grid during query execution"))
+                    return;
+            }
+
+            if (cacheEx != null && cacheEx.getMessage() != null) {
+                if (cacheEx.getMessage().contains("Query was interrupted."))
+                    return;
+            }
+
+            if (cacheEx != null && cacheEx.getMessage() != null) {
+                if (cacheEx.getMessage().contains("Failed to fetch data from node"))
+                    return;
+            }
+
+            if (cacheEx != null && cacheEx.getMessage() != null) {
+                if (cacheEx.getMessage().contains("Failed to send message"))
+                    return;
+            }
+        }
+
+        if (X.hasCause(e, IgniteSQLException.class)) {
+            IgniteSQLException sqlEx = X.cause(e, IgniteSQLException.class);
+
+            if (sqlEx != null && sqlEx.getMessage() != null) {
+                if (sqlEx.getMessage().contains("Transaction is already completed."))
+                    return;
+
+                if (sqlEx.getMessage().contains("Cannot serialize transaction due to write conflict"))
+                    return;
+            }
+        }
+
+        fail("Unexpected tx exception. " + X.getFullStackTrace(e));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java
index e583546..99d0894 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.database;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
@@ -27,6 +28,7 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow;
 import org.apache.ignite.internal.transactions.IgniteTxMvccVersionCheckedException;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
 
@@ -47,17 +49,22 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<GridH2SearchRo
     /** */
     private final GridCacheContext cctx;
 
+    /** */
+    private final IgniteLogger log;
+
     /**
      * @param filter Cache filter.
      * @param mvccSnapshot MVCC snapshot.
      * @param cctx Cache context.
      */
-    public H2TreeFilterClosure(IndexingQueryCacheFilter filter, MvccSnapshot mvccSnapshot, GridCacheContext cctx) {
+    public H2TreeFilterClosure(IndexingQueryCacheFilter filter, MvccSnapshot mvccSnapshot, GridCacheContext cctx,
+        IgniteLogger log) {
         assert (filter != null || mvccSnapshot != null) && cctx != null ;
 
         this.filter = filter;
         this.mvccSnapshot = mvccSnapshot;
         this.cctx = cctx;
+        this.log = log;
     }
 
     /** {@inheritDoc} */
@@ -97,8 +104,14 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<GridH2SearchRo
         try {
             return isVisible(cctx, mvccSnapshot, rowCrdVer, rowCntr, rowOpCntr, io.getLink(pageAddr, idx));
         }
-        catch (IgniteTxMvccVersionCheckedException ignored) {
-            return false; // The row is going to be removed.
+        catch (IgniteTxMvccVersionCheckedException e) {
+            // We expect the active tx state can be observed by read tx only in the cases when tx has been aborted
+            // asynchronously and node hasn't received finish message yet but coordinator has already removed it from
+            // the active txs map. Rows written by this tx are invisible to anyone and will be removed by the vacuum.
+            if (log.isDebugEnabled())
+                log.debug( "Unexpected tx state on index lookup. " + X.getFullStackTrace(e));
+
+            return false;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index a15742f..e9cca9e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -466,7 +466,7 @@ public class H2TreeIndex extends GridH2IndexBase {
         if(p == null && v == null)
             return null;
 
-        return new H2TreeFilterClosure(p, v, cctx);
+        return new H2TreeFilterClosure(p, v, cctx, log);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 62c5c78..b30aa2f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -1202,9 +1202,6 @@ public class GridReduceQueryExecutor {
      */
     public void releaseRemoteResources(Collection<ClusterNode> nodes, ReduceQueryRun r, long qryReqId,
         boolean distributedJoins, MvccQueryTracker mvccTracker) {
-        if (mvccTracker != null)
-            mvccTracker.onDone();
-
         // For distributedJoins need always send cancel request to cleanup resources.
         if (distributedJoins)
             send(nodes, new GridQueryCancelRequest(qryReqId), null, false);
@@ -1220,6 +1217,8 @@ public class GridReduceQueryExecutor {
 
         if (!runs.remove(qryReqId, r))
             U.warn(log, "Query run was already removed: " + qryReqId);
+        else if (mvccTracker != null)
+            mvccTracker.onDone();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d76ff542/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java
index 1362b4a..b347497 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.CacheMode;
 import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN;
 import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL;
 import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML;
+import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.PUT;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
@@ -63,6 +64,14 @@ public class CacheMvccPartitionedSqlCoordinatorFailoverTest extends CacheMvccAbs
     /**
      * @throws Exception If failed.
      */
+    public void testPutAllGetAll_ClientServer_Backups1_Restart_Scan() throws Exception {
+        // TODO add tests with RESTART_RND_SRV https://issues.apache.org/jira/browse/IGNITE-9928
+        putAllGetAll(RestartMode.RESTART_RND_SRV, 4, 2, 2, 64, /*new InitIndexing(Integer.class, Integer.class)*/ null, SCAN, PUT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testPutAllGetAll_ClientServer_Backups1_RestartCoordinator_ScanDml_Persistence() throws Exception {
         persistence = true;
 


[34/50] [abbrv] ignite git commit: IGNITE-9738 Client node can suddenly fail on start - Fixes #4968.

Posted by sb...@apache.org.
IGNITE-9738 Client node can suddenly fail on start - Fixes #4968.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d82b21ec
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d82b21ec
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d82b21ec

Branch: refs/heads/ignite-gg-14206
Commit: d82b21ec56a956fa7cc5374e3f15e279e7c492ac
Parents: e1f8f46
Author: vd-pyatkov <vp...@gridgain.com>
Authored: Mon Oct 22 18:19:53 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Oct 22 18:19:53 2018 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  10 +-
 .../LongClientConnectToClusterTest.java         | 173 +++++++++++++++++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   2 +
 3 files changed, 180 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d82b21ec/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index d3a8b18..92c197a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -305,6 +305,11 @@ class ClientImpl extends TcpDiscoveryImpl {
             }
         }.start();
 
+        timer.schedule(
+            new MetricsSender(),
+            spi.metricsUpdateFreq,
+            spi.metricsUpdateFreq);
+
         try {
             joinLatch.await();
 
@@ -317,11 +322,6 @@ class ClientImpl extends TcpDiscoveryImpl {
             throw new IgniteSpiException("Thread has been interrupted.", e);
         }
 
-        timer.schedule(
-            new MetricsSender(),
-            spi.metricsUpdateFreq,
-            spi.metricsUpdateFreq);
-
         spi.printStartInfo();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d82b21ec/modules/core/src/test/java/org/apache/ignite/spi/discovery/LongClientConnectToClusterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/LongClientConnectToClusterTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/LongClientConnectToClusterTest.java
new file mode 100644
index 0000000..a079926
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/LongClientConnectToClusterTest.java
@@ -0,0 +1,173 @@
+/*
+ * 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.spi.discovery;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Test client connects to two nodes cluster during time more than the
+ * {@link org.apache.ignite.configuration.IgniteConfiguration#clientFailureDetectionTimeout}.
+ */
+public class LongClientConnectToClusterTest extends GridCommonAbstractTest {
+    /** Client instance name. */
+    public static final String CLIENT_INSTANCE_NAME = "client";
+    /** Client metrics update count. */
+    private static volatile int clientMetricsUpdateCnt;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        TcpDiscoverySpi discoSpi = getTestIgniteInstanceName(0).equals(igniteInstanceName)
+            ? new DelayedTcpDiscoverySpi()
+            : getTestIgniteInstanceName(1).equals(igniteInstanceName)
+            ? new UpdateMetricsInterceptorTcpDiscoverySpi()
+            : new TcpDiscoverySpi();
+
+        return super.getConfiguration(igniteInstanceName)
+            .setClientMode(igniteInstanceName.startsWith(CLIENT_INSTANCE_NAME))
+            .setClientFailureDetectionTimeout(1_000)
+            .setMetricsUpdateFrequency(500)
+            .setDiscoverySpi(discoSpi
+                .setReconnectCount(1)
+                .setLocalAddress("127.0.0.1")
+                .setIpFinder(new TcpDiscoveryVmIpFinder()
+                    .setAddresses(Collections.singletonList(igniteInstanceName.startsWith(CLIENT_INSTANCE_NAME)
+                        ? "127.0.0.1:47501"
+                        : "127.0.0.1:47500..47502"))));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Test method.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientConnectToCluster() throws Exception {
+        clientMetricsUpdateCnt = 0;
+
+        IgniteEx client = startGrid(CLIENT_INSTANCE_NAME);
+
+        assertTrue(clientMetricsUpdateCnt > 0);
+
+        assertTrue(client.localNode().isClient());
+
+        assertEquals(client.cluster().nodes().size(), 3);
+    }
+
+    /** Discovery SPI which intercept TcpDiscoveryClientMetricsUpdateMessage. */
+    private static class UpdateMetricsInterceptorTcpDiscoverySpi extends TcpDiscoverySpi {
+        /** */
+        private class DiscoverySpiListenerWrapper implements DiscoverySpiListener {
+            /** */
+            private DiscoverySpiListener delegate;
+
+            /**
+             * @param delegate Delegate.
+             */
+            private DiscoverySpiListenerWrapper(DiscoverySpiListener delegate) {
+                this.delegate = delegate;
+            }
+
+            /** {@inheritDoc} */
+            @Override public IgniteFuture<?> onDiscovery(
+                int type,
+                long topVer,
+                ClusterNode node,
+                Collection<ClusterNode> topSnapshot,
+                @Nullable Map<Long, Collection<ClusterNode>> topHist,
+                @Nullable DiscoverySpiCustomMessage spiCustomMsg
+            ) {
+                if (EventType.EVT_NODE_METRICS_UPDATED == type) {
+                    log.info("Metrics update message catched from node " + node);
+
+                    assertFalse(locNode.isClient());
+
+                    if (node.isClient())
+                        clientMetricsUpdateCnt++;
+                }
+
+                if (delegate != null)
+                    return delegate.onDiscovery(type, topVer, node, topSnapshot, topHist, spiCustomMsg);
+
+                return new IgniteFinishedFutureImpl<>();
+            }
+
+            /** {@inheritDoc} */
+            @Override public void onLocalNodeInitialized(ClusterNode locNode) {
+                if (delegate != null)
+                    delegate.onLocalNodeInitialized(locNode);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
+            super.setListener(new DiscoverySpiListenerWrapper(lsnr));
+        }
+    }
+
+    /** Discovery SPI delayed TcpDiscoveryNodeAddFinishedMessage. */
+    private static class DelayedTcpDiscoverySpi extends TcpDiscoverySpi {
+        /** Delay message period millis. */
+        public static final int DELAY_MSG_PERIOD_MILLIS = 2_000;
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(ClusterNode node, Socket sock, OutputStream out,
+            TcpDiscoveryAbstractMessage msg, long timeout) throws IOException, IgniteCheckedException {
+            if (msg instanceof TcpDiscoveryNodeAddFinishedMessage && msg.topologyVersion() == 3) {
+                log.info("Catched discovery message: " + msg);
+
+                try {
+                    Thread.sleep(DELAY_MSG_PERIOD_MILLIS);
+                }
+                catch (InterruptedException e) {
+                    log.error("Interrupt on DelayedTcpDiscoverySpi.", e);
+
+                    Thread.currentThread().interrupt();
+                }
+            }
+
+            super.writeToSocket(node, sock, out, msg, timeout);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d82b21ec/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index 04869f9..80f093d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -23,6 +23,7 @@ import org.apache.ignite.spi.GridTcpSpiForwardingSelfTest;
 import org.apache.ignite.spi.discovery.AuthenticationRestartTest;
 import org.apache.ignite.spi.discovery.FilterDataForClientNodeDiscoveryTest;
 import org.apache.ignite.spi.discovery.IgniteDiscoveryCacheReuseSelfTest;
+import org.apache.ignite.spi.discovery.LongClientConnectToClusterTest;
 import org.apache.ignite.spi.discovery.tcp.DiscoveryUnmarshalVulnerabilityTest;
 import org.apache.ignite.spi.discovery.tcp.IgniteClientConnectTest;
 import org.apache.ignite.spi.discovery.tcp.IgniteClientReconnectMassiveShutdownTest;
@@ -97,6 +98,7 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridTcpSpiForwardingSelfTest.class));
 
         suite.addTest(new TestSuite(TcpClientDiscoverySpiSelfTest.class));
+        suite.addTest(new TestSuite(LongClientConnectToClusterTest.class));
         suite.addTest(new TestSuite(TcpClientDiscoveryMarshallerCheckSelfTest.class));
         suite.addTest(new TestSuite(TcpClientDiscoverySpiMulticastTest.class));
         suite.addTest(new TestSuite(TcpClientDiscoverySpiFailureTimeoutSelfTest.class));


[12/50] [abbrv] ignite git commit: IGNITE-9854 Correct remove from dirtyPages and segCheckpointPages - Fixes #4988.

Posted by sb...@apache.org.
IGNITE-9854 Correct remove from dirtyPages and segCheckpointPages - Fixes #4988.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/829dc1f2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/829dc1f2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/829dc1f2

Branch: refs/heads/ignite-gg-14206
Commit: 829dc1f240c07731a1ee98ae18c80ea6074dc6c4
Parents: a9f37a2
Author: Ivan Daschinskiy <iv...@gmail.com>
Authored: Thu Oct 18 14:33:28 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Thu Oct 18 14:33:28 2018 +0300

----------------------------------------------------------------------
 .../cache/persistence/pagemem/PageMemoryImpl.java   | 16 ++++++++++------
 1 file changed, 10 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/829dc1f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 55c5fb5..48e33c0 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -851,11 +851,15 @@ public class PageMemoryImpl implements PageMemoryEx {
         if (rmv)
             seg.loadedPages.remove(grpId, PageIdUtils.effectivePageId(pageId));
 
-        if (seg.segCheckpointPages != null)
-            seg.segCheckpointPages.remove(new FullPageId(pageId, grpId));
+        Collection<FullPageId> cpPages = seg.segCheckpointPages;
 
-        if (seg.dirtyPages != null)
-            seg.dirtyPages.remove(new FullPageId(pageId, grpId));
+        if (cpPages != null)
+            cpPages.remove(new FullPageId(pageId, grpId));
+
+        Collection<FullPageId> dirtyPages = seg.dirtyPages;
+
+        if (dirtyPages != null)
+            dirtyPages.remove(new FullPageId(pageId, grpId));
 
         return relPtr;
     }
@@ -1875,7 +1879,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         private static final int ACQUIRED_PAGES_PADDING = 4;
 
         /** Page ID to relative pointer map. */
-        private LoadedPagesMap loadedPages;
+        private final LoadedPagesMap loadedPages;
 
         /** Pointer to acquired pages integer counter. */
         private long acquiredPagesPtr;
@@ -1887,7 +1891,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         private long memPerTbl;
 
         /** Pages marked as dirty since the last checkpoint. */
-        private Collection<FullPageId> dirtyPages = new GridConcurrentHashSet<>();
+        private volatile Collection<FullPageId> dirtyPages = new GridConcurrentHashSet<>();
 
         /** */
         private volatile Collection<FullPageId> segCheckpointPages;


[49/50] [abbrv] ignite git commit: Revert "IGNITE-5795 Register binary metadata during cache start - Fixes #4852."

Posted by sb...@apache.org.
Revert "IGNITE-5795 Register binary metadata during cache start - Fixes #4852."

This reverts commit 3bb03444246f863096063d084393676a84d2bc0e.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/86f54371
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/86f54371
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/86f54371

Branch: refs/heads/ignite-gg-14206
Commit: 86f543713b148541a6d497c4ab6404e7e7edf815
Parents: 324e6e3
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Oct 23 17:54:24 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Oct 23 17:54:24 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../binary/BinaryCachingMetadataHandler.java    |  25 +-
 .../ignite/internal/binary/BinaryContext.java   |  59 +---
 .../internal/binary/BinaryMetadataHandler.java  |  10 -
 .../binary/BinaryNoopMetadataHandler.java       |   6 -
 .../binary/builder/BinaryObjectBuilderImpl.java |   2 +-
 .../internal/client/thin/TcpIgniteClient.java   |   6 -
 .../processors/cache/GridCacheProcessor.java    |   9 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   5 -
 .../processors/query/GridQueryProcessor.java    |  77 +----
 .../binary/TestCachingMetadataHandler.java      |   6 -
 .../cache/CacheRegisterMetadataLocallyTest.java | 287 -------------------
 .../cache/index/AbstractSchemaSelfTest.java     |   6 +-
 .../index/H2DynamicIndexAbstractSelfTest.java   |  48 ++--
 .../IgniteCacheWithIndexingTestSuite.java       |   2 -
 15 files changed, 56 insertions(+), 494 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 1546da3..1f9a5e8 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
@@ -1005,7 +1005,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Start processors before discovery manager, so they will
             // be able to start receiving messages once discovery completes.
             try {
-                startProcessor(new GridMarshallerMappingProcessor(ctx));
                 startProcessor(new PdsConsistentIdProcessor(ctx));
                 startProcessor(new MvccProcessorImpl(ctx));
                 startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx));
@@ -1029,6 +1028,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 startProcessor(createHadoopComponent());
                 startProcessor(new DataStructuresProcessor(ctx));
                 startProcessor(createComponent(PlatformProcessor.class, ctx));
+                startProcessor(new GridMarshallerMappingProcessor(ctx));
 
                 // Start plugins.
                 for (PluginProvider provider : ctx.plugins().allProviders()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
index b60dc097..a0559cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
@@ -46,29 +46,24 @@ public class BinaryCachingMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void addMeta(int typeId, BinaryType type,
-        boolean failIfUnregistered) throws BinaryObjectException {
-        BinaryType oldType = metas.put(typeId, type);
+    @Override public synchronized void addMeta(int typeId, BinaryType type, boolean failIfUnregistered) throws BinaryObjectException {
+        synchronized (this) {
+            BinaryType oldType = metas.put(typeId, type);
 
-        if (oldType != null) {
-            BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
-            BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
+            if (oldType != null) {
+                BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
+                BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
 
-            BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta);
+                BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta);
 
-            BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
+                BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
 
-            metas.put(typeId, mergedType);
+                metas.put(typeId, mergedType);
+            }
         }
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
-        throws BinaryObjectException {
-        addMeta(typeId, meta, failIfUnregistered);
-    }
-
-    /** {@inheritDoc} */
     @Override public synchronized BinaryType metadata(int typeId) throws BinaryObjectException {
         return metas.get(typeId);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 7ab74e0..7885d95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -617,18 +617,6 @@ public class BinaryContext {
      */
     public BinaryClassDescriptor descriptorForClass(Class<?> cls, boolean deserialize, boolean failIfUnregistered)
         throws BinaryObjectException {
-        return descriptorForClass(cls, deserialize, failIfUnregistered, false);
-    }
-
-    /**
-     * @param cls Class.
-     * @param failIfUnregistered Throw exception if class isn't registered.
-     * @param onlyLocReg {@code true} if descriptor need to register only locally when registration is required at all.
-     * @return Class descriptor.
-     * @throws BinaryObjectException In case of error.
-     */
-    public BinaryClassDescriptor descriptorForClass(Class<?> cls, boolean deserialize, boolean failIfUnregistered,
-        boolean onlyLocReg) throws BinaryObjectException {
         assert cls != null;
 
         BinaryClassDescriptor desc = descByCls.get(cls);
@@ -637,7 +625,7 @@ public class BinaryContext {
             if (failIfUnregistered)
                 throw new UnregisteredClassException(cls);
 
-            desc = registerClassDescriptor(cls, deserialize, onlyLocReg);
+            desc = registerClassDescriptor(cls, deserialize);
         }
         else if (!desc.registered()) {
             if (!desc.userType()) {
@@ -674,7 +662,7 @@ public class BinaryContext {
                 if (failIfUnregistered)
                     throw new UnregisteredClassException(cls);
 
-                desc = registerUserClassDescriptor(desc, onlyLocReg);
+                desc = registerUserClassDescriptor(desc);
             }
         }
 
@@ -727,7 +715,7 @@ public class BinaryContext {
         }
 
         if (desc == null) {
-            desc = registerClassDescriptor(cls, deserialize, false);
+            desc = registerClassDescriptor(cls, deserialize);
 
             assert desc.typeId() == typeId : "Duplicate typeId [typeId=" + typeId + ", cls=" + cls
                 + ", desc=" + desc + "]";
@@ -740,10 +728,9 @@ public class BinaryContext {
      * Creates and registers {@link BinaryClassDescriptor} for the given {@code class}.
      *
      * @param cls Class.
-     * @param onlyLocReg {@code true} if descriptor need to register only locally when registration is required at all.
      * @return Class descriptor.
      */
-    private BinaryClassDescriptor registerClassDescriptor(Class<?> cls, boolean deserialize, boolean onlyLocReg) {
+    private BinaryClassDescriptor registerClassDescriptor(Class<?> cls, boolean deserialize) {
         BinaryClassDescriptor desc;
 
         String clsName = cls.getName();
@@ -772,7 +759,7 @@ public class BinaryContext {
                 desc = old;
         }
         else
-            desc = registerUserClassDescriptor(cls, deserialize, onlyLocReg);
+            desc = registerUserClassDescriptor(cls, deserialize);
 
         return desc;
     }
@@ -781,10 +768,9 @@ public class BinaryContext {
      * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}.
      *
      * @param cls Class.
-     * @param onlyLocReg {@code true} if descriptor need to register only locally.
      * @return Class descriptor.
      */
-    private BinaryClassDescriptor registerUserClassDescriptor(Class<?> cls, boolean deserialize, boolean onlyLocReg) {
+    private BinaryClassDescriptor registerUserClassDescriptor(Class<?> cls, boolean deserialize) {
         boolean registered;
 
         final String clsName = cls.getName();
@@ -795,7 +781,7 @@ public class BinaryContext {
 
         final int typeId = mapper.typeId(clsName);
 
-        registered = registerUserClassName(typeId, cls.getName(), onlyLocReg);
+        registered = registerUserClassName(typeId, cls.getName());
 
         BinarySerializer serializer = serializerForClass(cls);
 
@@ -813,22 +799,9 @@ public class BinaryContext {
             registered
         );
 
-        if (!deserialize) {
-            BinaryMetadata binaryMetadata = new BinaryMetadata(
-                typeId,
-                typeName,
-                desc.fieldsMeta(),
-                affFieldName,
-                null,
-                desc.isEnum(),
-                cls.isEnum() ? enumMap(cls) : null
-            );
-
-            if (onlyLocReg)
-                metaHnd.addMetaLocally(typeId, binaryMetadata.wrap(this), false);
-            else
-                metaHnd.addMeta(typeId, binaryMetadata.wrap(this), false);
-        }
+        if (!deserialize)
+            metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, null,
+                desc.isEnum(), cls.isEnum() ? enumMap(cls) : null).wrap(this), false);
 
         descByCls.put(cls, desc);
 
@@ -841,13 +814,12 @@ public class BinaryContext {
      * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}.
      *
      * @param desc Old descriptor that should be re-registered.
-     * @param onlyLocReg {@code true} if descriptor need to register only locally.
      * @return Class descriptor.
      */
-    private BinaryClassDescriptor registerUserClassDescriptor(BinaryClassDescriptor desc, boolean onlyLocReg) {
+    private BinaryClassDescriptor registerUserClassDescriptor(BinaryClassDescriptor desc) {
         boolean registered;
 
-        registered = registerUserClassName(desc.typeId(), desc.describedClass().getName(), onlyLocReg);
+        registered = registerUserClassName(desc.typeId(), desc.describedClass().getName());
 
         if (registered) {
             BinarySerializer serializer = desc.initialSerializer();
@@ -1219,18 +1191,15 @@ public class BinaryContext {
      *
      * @param typeId Type ID.
      * @param clsName Class Name.
-     * @param onlyLocReg {@code true} if descriptor need to register only locally.
      * @return {@code True} if the mapping was registered successfully.
      */
-    public boolean registerUserClassName(int typeId, String clsName, boolean onlyLocReg) {
+    public boolean registerUserClassName(int typeId, String clsName) {
         IgniteCheckedException e = null;
 
         boolean res = false;
 
         try {
-            res = onlyLocReg
-                ? marshCtx.registerClassNameLocally(JAVA_ID, typeId, clsName)
-                : marshCtx.registerClassName(JAVA_ID, typeId, clsName);
+            res = marshCtx.registerClassName(JAVA_ID, typeId, clsName);
         }
         catch (DuplicateTypeIdException dupEx) {
             // Ignore if trying to register mapped type name of the already registered class name and vise versa

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
index d1336bf..85ab137 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
@@ -36,16 +36,6 @@ public interface BinaryMetadataHandler {
     public void addMeta(int typeId, BinaryType meta, boolean failIfUnregistered) throws BinaryObjectException;
 
     /**
-     * Adds meta data locally on current node without sending any messages.
-     *
-     * @param typeId Type ID.
-     * @param meta Metadata.
-     * @param failIfUnregistered Fail if unregistered.
-     * @throws BinaryObjectException In case of error.
-     */
-    public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered) throws BinaryObjectException;
-
-    /**
      * Gets meta data for provided type ID.
      *
      * @param typeId Type ID.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
index a552d61..4ee2428 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
@@ -48,12 +48,6 @@ public class BinaryNoopMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
-        throws BinaryObjectException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index 5414a25..abd63cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -364,7 +364,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                 if (affFieldName0 == null)
                     affFieldName0 = ctx.affinityKeyFieldName(typeId);
 
-                ctx.registerUserClassName(typeId, typeName, false);
+                ctx.registerUserClassName(typeId, typeName);
 
                 ctx.updateMetadata(typeId, new BinaryMetadata(typeId, typeName, fieldsMeta, affFieldName0,
                     Collections.singleton(curSchema), false, null), writer.failIfUnregistered());

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
index 856f41c..5040816 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
@@ -263,12 +263,6 @@ public class TcpIgniteClient implements IgniteClient {
         }
 
         /** {@inheritDoc} */
-        @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
-            throws BinaryObjectException {
-            throw new UnsupportedOperationException("Can't register metadata locally for thin client.");
-        }
-
-        /** {@inheritDoc} */
         @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
             BinaryType meta = cache.metadata(typeId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 59703c0..4f8301e 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
@@ -4066,13 +4066,8 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
             return msg0.needExchange();
         }
 
-        if (msg instanceof DynamicCacheChangeBatch) {
-            boolean changeRequested = cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer);
-
-            ctx.query().onCacheChangeRequested((DynamicCacheChangeBatch)msg);
-
-            return changeRequested;
-        }
+        if (msg instanceof DynamicCacheChangeBatch)
+            return cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer);
 
         if (msg instanceof DynamicCacheChangeFailureMessage)
             cachesInfo.onCacheChangeRequested((DynamicCacheChangeFailureMessage)msg, topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index f3078cb..137db9f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -207,11 +207,6 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                     CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(binaryCtx), failIfUnregistered);
                 }
 
-                @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
-                    throws BinaryObjectException {
-                    CacheObjectBinaryProcessorImpl.this.addMetaLocally(typeId, meta);
-                }
-
                 @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
                     return CacheObjectBinaryProcessorImpl.this.metadata(typeId);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index ea588ff..43310a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import javax.cache.Cache;
-import javax.cache.CacheException;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -36,6 +34,8 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.Cache;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -62,20 +62,16 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
-import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
-import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
@@ -261,8 +257,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     ctxs.queries().evictDetailMetrics();
             }
         }, QRY_DETAIL_METRICS_EVICTION_FREQ, QRY_DETAIL_METRICS_EVICTION_FREQ);
-
-        registerMetadataForRegisteredCaches();
     }
 
     /** {@inheritDoc} */
@@ -910,73 +904,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Register metadata locally for already registered caches.
-     */
-    private void registerMetadataForRegisteredCaches() {
-        for (DynamicCacheDescriptor cacheDescriptor : ctx.cache().cacheDescriptors().values()) {
-            registerBinaryMetadata(cacheDescriptor.cacheConfiguration(), cacheDescriptor.schema());
-        }
-    }
-
-    /**
-     * Handle of cache change request.
-     *
-     * @param batch Dynamic cache change batch request.
-     */
-    public void onCacheChangeRequested(DynamicCacheChangeBatch batch) {
-        for (DynamicCacheChangeRequest req : batch.requests()) {
-            if (!req.start())
-                continue;
-
-            registerBinaryMetadata(req.startCacheConfiguration(), req.schema());
-        }
-    }
-
-    /**
-     * Register binary metadata locally.
-     *
-     * @param ccfg Cache configuration.
-     * @param schema Schema for which register metadata is required.
-     */
-    private void registerBinaryMetadata(CacheConfiguration ccfg, QuerySchema schema) {
-        if (schema != null) {
-            Collection<QueryEntity> qryEntities = schema.entities();
-
-            if (!F.isEmpty(qryEntities)) {
-                boolean binaryEnabled = ctx.cacheObjects().isBinaryEnabled(ccfg);
-
-                if (binaryEnabled) {
-                    for (QueryEntity qryEntity : qryEntities) {
-                        Class<?> keyCls = U.box(U.classForName(qryEntity.findKeyType(), null, true));
-                        Class<?> valCls = U.box(U.classForName(qryEntity.findValueType(), null, true));
-
-                        if (keyCls != null)
-                            registerDescriptorLocallyIfNeeded(keyCls);
-
-                        if (valCls != null)
-                            registerDescriptorLocallyIfNeeded(valCls);
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Register class metadata locally if it didn't do it earlier.
-     *
-     * @param cls Class for which the metadata should be registered.
-     */
-    private void registerDescriptorLocallyIfNeeded(Class<?> cls) {
-        IgniteCacheObjectProcessor cacheObjProc = ctx.cacheObjects();
-
-        if (cacheObjProc instanceof CacheObjectBinaryProcessorImpl) {
-            ((CacheObjectBinaryProcessorImpl)cacheObjProc)
-                .binaryContext()
-                .descriptorForClass(cls, false, false, true);
-        }
-    }
-
-    /**
      * Handle custom discovery message.
      *
      * @param msg Message.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
index 47138dd..c515f81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
@@ -39,12 +39,6 @@ public class TestCachingMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
-        throws BinaryObjectException {
-        addMeta(typeId, meta, failIfUnregistered);
-    }
-
-    /** {@inheritDoc} */
     @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
         return metas.get(typeId);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java
deleted file mode 100644
index d4066c2..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * 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.cache;
-
-import java.util.Collections;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.affinity.AffinityKeyMapped;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.managers.communication.GridIoMessage;
-import org.apache.ignite.internal.managers.discovery.CustomMessageWrapper;
-import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage;
-import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage;
-import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateProposedMessage;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.plugin.extensions.communication.Message;
-import org.apache.ignite.spi.IgniteSpiException;
-import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Tests, that binary metadata is registered correctly during the start without extra request to grid.
- */
-public class CacheRegisterMetadataLocallyTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final String STATIC_CACHE_NAME = "staticCache";
-
-    /** */
-    private static final String DYNAMIC_CACHE_NAME = "dynamicCache";
-
-    /** Holder of sent custom messages. */
-    private final ConcurrentLinkedQueue<Object> customMessages = new ConcurrentLinkedQueue<>();
-
-    /** Holder of sent communication messages. */
-    private final ConcurrentLinkedQueue<Object> communicationMessages = new ConcurrentLinkedQueue<>();
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setDiscoverySpi(new TcpDiscoverySpi() {
-            @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
-                if (msg instanceof CustomMessageWrapper)
-                    customMessages.add(((CustomMessageWrapper)msg).delegate());
-                else
-                    customMessages.add(msg);
-
-                super.sendCustomEvent(msg);
-            }
-        });
-
-        cfg.setCommunicationSpi(new TcpCommunicationSpi() {
-            @Override public void sendMessage(ClusterNode node, Message msg,
-                IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
-                if (msg instanceof GridIoMessage)
-                    communicationMessages.add(((GridIoMessage)msg).message());
-
-                super.sendMessage(node, msg, ackC);
-            }
-
-            @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
-                if (msg instanceof GridIoMessage)
-                    communicationMessages.add(((GridIoMessage)msg).message());
-
-                super.sendMessage(node, msg);
-            }
-        });
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
-
-        if (igniteInstanceName.equals("client"))
-            cfg.setClientMode(true);
-
-        cfg.setCacheConfiguration(cacheConfiguration(STATIC_CACHE_NAME, StaticKey.class, StaticValue.class));
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-
-        cleanPersistenceDir();
-
-        customMessages.clear();
-        communicationMessages.clear();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityKeyRegisteredStaticCache() throws Exception {
-        Ignite ignite = startGrid();
-
-        assertEquals("affKey", getAffinityKey(ignite, StaticKey.class));
-        assertEquals("affKey", getAffinityKey(ignite, StaticValue.class));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityKeyRegisteredDynamicCache() throws Exception {
-        Ignite ignite = startGrid();
-
-        ignite.createCache(cacheConfiguration(DYNAMIC_CACHE_NAME, DynamicKey.class, DynamicValue.class));
-
-        assertEquals("affKey", getAffinityKey(ignite, DynamicKey.class));
-        assertEquals("affKey", getAffinityKey(ignite, DynamicValue.class));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientFindsValueByAffinityKeyStaticCacheWithoutExtraRequest() throws Exception {
-        Ignite srv = startGrid();
-        IgniteCache<StaticKey, StaticValue> cache = srv.cache(STATIC_CACHE_NAME);
-
-        testClientFindsValueByAffinityKey(cache, new StaticKey(1), new StaticValue(2));
-
-        assertCustomMessages(2); //MetadataUpdateProposedMessage for update schema.
-        assertCommunicationMessages();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientFindsValueByAffinityKeyDynamicCacheWithoutExtraRequest() throws Exception {
-        Ignite srv = startGrid();
-        IgniteCache<DynamicKey, DynamicValue> cache =
-            srv.createCache(cacheConfiguration(DYNAMIC_CACHE_NAME, DynamicKey.class, DynamicValue.class));
-
-        testClientFindsValueByAffinityKey(cache, new DynamicKey(3), new DynamicValue(4));
-
-        //Expected only DynamicCacheChangeBatch for start cache and MetadataUpdateProposedMessage for update schema.
-        assertCustomMessages(3);
-        assertCommunicationMessages();
-    }
-
-    /**
-     * @param ignite Ignite instance.
-     * @param keyCls Key class.
-     * @return Name of affinity key field of the given class.
-     */
-    private <K> String getAffinityKey(Ignite ignite, Class<K> keyCls) {
-        BinaryType binType = ignite.binary().type(keyCls);
-
-        return binType.affinityKeyFieldName();
-    }
-
-    /**
-     * @param cache Cache instance.
-     * @param key Test key.
-     * @param val Test value.
-     * @throws Exception If failed.
-     */
-    private <K, V> void testClientFindsValueByAffinityKey(IgniteCache<K, V> cache, K key, V val) throws Exception {
-        cache.put(key, val);
-
-        assertTrue(cache.containsKey(key));
-
-        Ignite client = startGrid("client");
-
-        IgniteCache<K, V> clientCache = client.cache(cache.getName());
-
-        assertTrue(clientCache.containsKey(key));
-    }
-
-    /**
-     * @param name Cache name.
-     * @param keyCls Key {@link Class}.
-     * @param valCls Value {@link Class}.
-     * @param <K> Key type.
-     * @param <V> Value type.
-     * @return Cache configuration
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, Class<K> keyCls, Class<V> valCls) {
-        CacheConfiguration<K, V> cfg = new CacheConfiguration<>(name);
-        cfg.setQueryEntities(Collections.singleton(new QueryEntity(keyCls, valCls)));
-        return cfg;
-    }
-
-    /**
-     * Expecting that "proposed binary metadata"( {@link org.apache.ignite.internal.processors.marshaller.MappingProposedMessage},
-     * {@link org.apache.ignite.internal.processors.cache.binary.MetadataUpdateProposedMessage}) will be skipped because
-     * it should be register locally during the start.
-     *
-     * @param expMsgCnt Count of expected messages.
-     */
-    private void assertCustomMessages(int expMsgCnt) {
-        assertEquals(customMessages.toString(), expMsgCnt, customMessages.size());
-
-        customMessages.forEach(cm -> assertTrue(cm.toString(), cm instanceof DynamicCacheChangeBatch || cm instanceof MetadataUpdateProposedMessage));
-    }
-
-    /**
-     * Expecting that extra request to binary metadata( {@link MetadataRequestMessage}, {@link MetadataResponseMessage})
-     * will be skipped because it should be register locally during the start.
-     */
-    private void assertCommunicationMessages() {
-        communicationMessages.forEach(cm ->
-            assertFalse(cm.toString(), cm instanceof MetadataRequestMessage || cm instanceof MetadataResponseMessage)
-        );
-    }
-
-    /** */
-    private static class StaticKey {
-        /** */
-        @AffinityKeyMapped
-        private int affKey;
-
-        /**
-         * @param affKey Affinity key.
-         */
-        StaticKey(int affKey) {
-            this.affKey = affKey;
-        }
-    }
-
-    /** */
-    private static class StaticValue {
-        /** */
-        @AffinityKeyMapped
-        private int affKey;
-
-        /**
-         * @param affKey Affinity key.
-         */
-        StaticValue(int affKey) {
-        }
-    }
-
-    /** */
-    private static class DynamicKey {
-        /** */
-        @AffinityKeyMapped
-        private int affKey;
-
-        /**
-         * @param affKey Affinity key.
-         */
-        DynamicKey(int affKey) {
-            this.affKey = affKey;
-        }
-    }
-
-    /** */
-    private static class DynamicValue {
-        /** */
-        @AffinityKeyMapped
-        private int affKey;
-
-        /**
-         * @param affKey Affinity key.
-         */
-        DynamicValue(int affKey) {
-            this.affKey = affKey;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
index 5f0e18e..7f1e2e7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
@@ -629,21 +629,21 @@ public abstract class AbstractSchemaSelfTest extends GridCommonAbstractTest {
     public static class ValueClass {
         /** Field 1. */
         @QuerySqlField
-        private Long field1;
+        private String field1;
 
         /**
          * Constructor.
          *
          * @param field1 Field 1.
          */
-        public ValueClass(Long field1) {
+        public ValueClass(String field1) {
             this.field1 = field1;
         }
 
         /**
          * @return Field 1
          */
-        public Long field1() {
+        public String field1() {
             return field1;
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
index bfc3881..2d7a636 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
@@ -60,9 +60,9 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
 
         IgniteCache<KeyClass, ValueClass> cache = client().cache(CACHE_NAME);
 
-        cache.put(new KeyClass(1), new ValueClass(1L));
-        cache.put(new KeyClass(2), new ValueClass(2L));
-        cache.put(new KeyClass(3), new ValueClass(3L));
+        cache.put(new KeyClass(1), new ValueClass("val1"));
+        cache.put(new KeyClass(2), new ValueClass("val2"));
+        cache.put(new KeyClass(3), new ValueClass("val3"));
     }
 
     /** {@inheritDoc} */
@@ -89,14 +89,14 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
                 continue;
 
             List<List<?>> locRes = ignite(i).cache("cache").query(new SqlFieldsQuery("explain select \"id\" from " +
-                "\"cache\".\"ValueClass\" where \"field1\" = 1").setLocal(true)).getAll();
+                "\"cache\".\"ValueClass\" where \"field1\" = 'A'").setLocal(true)).getAll();
 
             assertEquals(F.asList(
                 Collections.singletonList("SELECT\n" +
                     "    \"id\"\n" +
                     "FROM \"cache\".\"ValueClass\"\n" +
-                    "    /* \"cache\".\"idx_1\": \"field1\" = 1 */\n" +
-                    "WHERE \"field1\" = 1")
+                    "    /* \"cache\".\"idx_1\": \"field1\" = 'A' */\n" +
+                    "WHERE \"field1\" = 'A'")
             ), locRes);
         }
 
@@ -106,7 +106,7 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
 
         assertSize(2);
 
-        cache.put(new KeyClass(4), new ValueClass(1L));
+        cache.put(new KeyClass(4), new ValueClass("someVal"));
 
         assertSize(3);
     }
@@ -162,14 +162,14 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
                 continue;
 
             List<List<?>> locRes = ignite(i).cache("cache").query(new SqlFieldsQuery("explain select \"id\" from " +
-                "\"cache\".\"ValueClass\" where \"field1\" = 1").setLocal(true)).getAll();
+                "\"cache\".\"ValueClass\" where \"field1\" = 'A'").setLocal(true)).getAll();
 
             assertEquals(F.asList(
                 Collections.singletonList("SELECT\n" +
                     "    \"id\"\n" +
                     "FROM \"cache\".\"ValueClass\"\n" +
                     "    /* \"cache\".\"ValueClass\".__SCAN_ */\n" +
-                    "WHERE \"field1\" = 1")
+                    "WHERE \"field1\" = 'A'")
             ), locRes);
         }
 
@@ -204,39 +204,38 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
     public void testIndexState() {
         IgniteCache<KeyClass, ValueClass> cache = cache();
 
-        assertColumnValues(1L, 2L, 3L);
+        assertColumnValues("val1", "val2", "val3");
 
         cache.query(new SqlFieldsQuery("CREATE INDEX \"" + IDX_NAME_1_ESCAPED + "\" ON \"" + TBL_NAME_ESCAPED + "\"(\""
             + FIELD_NAME_1_ESCAPED + "\" ASC)"));
 
-        assertColumnValues(1L, 2L, 3L);
+        assertColumnValues("val1", "val2", "val3");
 
         cache.remove(new KeyClass(2));
 
-        assertColumnValues(1L, 3L);
+        assertColumnValues("val1", "val3");
 
-        cache.put(new KeyClass(0), new ValueClass(0L));
+        cache.put(new KeyClass(0), new ValueClass("someVal"));
 
-        assertColumnValues(0L, 1L, 3L);
+        assertColumnValues("someVal", "val1", "val3");
 
         cache.query(new SqlFieldsQuery("DROP INDEX \"" + IDX_NAME_1_ESCAPED + "\""));
 
-        assertColumnValues(0L, 1L, 3L);
+        assertColumnValues("someVal", "val1", "val3");
     }
 
     /**
      * Check that values of {@code field1} match what we expect.
      * @param vals Expected values.
      */
-    private void assertColumnValues(Long... vals) {
+    private void assertColumnValues(String... vals) {
         List<List<?>> expRes = new ArrayList<>(vals.length);
 
-        for (Long v : vals)
+        for (String v : vals)
             expRes.add(Collections.singletonList(v));
 
-        List<List<?>> all = cache().query(new SqlFieldsQuery("SELECT \"" + FIELD_NAME_1_ESCAPED + "\" FROM \"" +
-            TBL_NAME_ESCAPED + "\" ORDER BY \"id\"")).getAll();
-        assertEquals(expRes, all);
+        assertEquals(expRes, cache().query(new SqlFieldsQuery("SELECT \"" + FIELD_NAME_1_ESCAPED + "\" FROM \"" +
+            TBL_NAME_ESCAPED + "\" ORDER BY \"id\"")).getAll());
     }
 
     /**
@@ -246,9 +245,8 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
     private void assertSize(long expSize) {
         assertEquals(expSize, cache().size());
 
-        Object actual = cache().query(new SqlFieldsQuery("SELECT COUNT(*) from \"ValueClass\""))
-            .getAll().get(0).get(0);
-        assertEquals(expSize, actual);
+        assertEquals(expSize, cache().query(new SqlFieldsQuery("SELECT COUNT(*) from \"ValueClass\""))
+            .getAll().get(0).get(0));
     }
 
     /**
@@ -315,8 +313,8 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
         entity.setValueType(ValueClass.class.getName());
 
         entity.addQueryField("id", Long.class.getName(), null);
-        entity.addQueryField(FIELD_NAME_1_ESCAPED, Long.class.getName(), null);
-        entity.addQueryField(FIELD_NAME_2_ESCAPED, Long.class.getName(), null);
+        entity.addQueryField(FIELD_NAME_1_ESCAPED, String.class.getName(), null);
+        entity.addQueryField(FIELD_NAME_2_ESCAPED, String.class.getName(), null);
 
         entity.setKeyFields(Collections.singleton("id"));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86f54371/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index fae196f..8517ebb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -19,7 +19,6 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.BinaryTypeMismatchLoggingTest;
-import org.apache.ignite.internal.processors.cache.CacheRegisterMetadataLocallyTest;
 import org.apache.ignite.internal.processors.cache.CacheBinaryKeyConcurrentQueryTest;
 import org.apache.ignite.internal.processors.cache.CacheConfigurationP2PTest;
 import org.apache.ignite.internal.processors.cache.CacheIndexStreamerTest;
@@ -81,7 +80,6 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite {
         suite.addTestSuite(CacheOperationsWithExpirationTest.class);
         suite.addTestSuite(CacheBinaryKeyConcurrentQueryTest.class);
         suite.addTestSuite(CacheQueryFilterExpiredTest.class);
-        suite.addTestSuite(CacheRegisterMetadataLocallyTest.class);
 
         suite.addTestSuite(ClientReconnectAfterClusterRestartTest.class);
 


[38/50] [abbrv] ignite git commit: IGNITE-9960: SQL: Reverted IGNITE-9171 and IGNITE-9864 until performance is fixed. This closes #5045.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 9b7d268..f228111 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -40,6 +40,7 @@ import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -57,12 +58,12 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheInvalidStateException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.CompoundLockFuture;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionsReservation;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
@@ -70,10 +71,8 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshalla
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.h2.H2ConnectionWrapper;
 import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
-import org.apache.ignite.internal.processors.query.h2.ObjectPoolReusable;
 import org.apache.ignite.internal.processors.query.h2.ResultSetEnlistFuture;
 import org.apache.ignite.internal.processors.query.h2.UpdateResult;
 import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode;
@@ -98,13 +97,13 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
-import org.h2.api.ErrorCode;
+import org.apache.ignite.thread.IgniteThread;
 import org.h2.command.Prepared;
 import org.h2.jdbc.JdbcResultSet;
-import org.h2.jdbc.JdbcSQLException;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_FORCE_LAZY_RESULT_SET;
 import static org.apache.ignite.cache.PartitionLossPolicy.READ_ONLY_SAFE;
 import static org.apache.ignite.cache.PartitionLossPolicy.READ_WRITE_SAFE;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
@@ -124,6 +123,9 @@ import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2V
 @SuppressWarnings("ForLoopReplaceableByForEach")
 public class GridMapQueryExecutor {
     /** */
+    public static final boolean FORCE_LAZY = IgniteSystemProperties.getBoolean(IGNITE_SQL_FORCE_LAZY_RESULT_SET);
+
+    /** */
     private IgniteLogger log;
 
     /** */
@@ -147,8 +149,8 @@ public class GridMapQueryExecutor {
     /** Busy lock for lazy workers. */
     private final GridSpinBusyLock lazyWorkerBusyLock = new GridSpinBusyLock();
 
-    /** Stop guard. */
-    private final AtomicBoolean stopGuard = new AtomicBoolean();
+    /** Lazy worker stop guard. */
+    private final AtomicBoolean lazyWorkerStopGuard = new AtomicBoolean();
 
     /**
      * @param busyLock Busy lock.
@@ -205,21 +207,18 @@ public class GridMapQueryExecutor {
     }
 
     /**
-     * Stop query map executor, cleanup resources.
+     * Cancel active lazy queries and prevent submit of new queries.
      */
-    public void stop() {
-        if (!stopGuard.compareAndSet(false, true))
+    public void cancelLazyWorkers() {
+        if (!lazyWorkerStopGuard.compareAndSet(false, true))
             return;
 
-        for (MapNodeResults res : qryRess.values())
-            res.cancelAll();
-
-        for (MapQueryLazyWorker w : lazyWorkers.values())
-            w.stop(true);
-
         lazyWorkerBusyLock.block();
 
-        assert lazyWorkers.isEmpty() : "Not cleaned lazy workers: " + lazyWorkers.size();
+        for (MapQueryLazyWorker worker : lazyWorkers.values())
+            worker.stop(false);
+
+        lazyWorkers.clear();
     }
 
     /**
@@ -260,7 +259,7 @@ public class GridMapQueryExecutor {
      * @return Busy lock for lazy workers to guard their operations with.
      */
     GridSpinBusyLock busyLock() {
-        return lazyWorkerBusyLock;
+        return busyLock;
     }
 
     /**
@@ -555,7 +554,6 @@ public class GridMapQueryExecutor {
     /**
      * @param node Node.
      * @param req Query request.
-     * @throws IgniteCheckedException On error.
      */
     private void onQueryRequest(final ClusterNode node, final GridH2QueryRequest req) throws IgniteCheckedException {
         int[] qryParts = req.queryPartitions();
@@ -568,14 +566,10 @@ public class GridMapQueryExecutor {
             req.isFlagSet(GridH2QueryRequest.FLAG_IS_LOCAL),
             req.isFlagSet(GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS));
 
-        final GridDhtTxLocalAdapter tx;
-
-        GridH2SelectForUpdateTxDetails txReq = req.txDetails();
-
         final boolean enforceJoinOrder = req.isFlagSet(GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER);
         final boolean explain = req.isFlagSet(GridH2QueryRequest.FLAG_EXPLAIN);
         final boolean replicated = req.isFlagSet(GridH2QueryRequest.FLAG_REPLICATED);
-        final boolean lazy = req.isFlagSet(GridH2QueryRequest.FLAG_LAZY) && txReq == null;
+        final boolean lazy = (FORCE_LAZY && req.queries().size() == 1) || req.isFlagSet(GridH2QueryRequest.FLAG_LAZY);
 
         final List<Integer> cacheIds = req.caches();
 
@@ -584,6 +578,10 @@ public class GridMapQueryExecutor {
 
         final Object[] params = req.parameters();
 
+        final GridDhtTxLocalAdapter tx;
+
+        GridH2SelectForUpdateTxDetails txReq = req.txDetails();
+
         try {
             if (txReq != null) {
                 // Prepare to run queries.
@@ -738,11 +736,7 @@ public class GridMapQueryExecutor {
      * @param parts Explicit partitions for current node.
      * @param pageSize Page size.
      * @param distributedJoinMode Query distributed join mode.
-     * @param enforceJoinOrder Enforce join order flag.
-     * @param replicated Replicated flag.
-     * @param timeout Query timeout.
-     * @param params Query params.
-     * @param lazy Lazy query execution flag.
+     * @param lazy Streaming flag.
      * @param mvccSnapshot MVCC snapshot.
      * @param tx Transaction.
      * @param txDetails TX details, if it's a {@code FOR UPDATE} request, or {@code null}.
@@ -771,24 +765,75 @@ public class GridMapQueryExecutor {
         @Nullable final GridH2SelectForUpdateTxDetails txDetails,
         @Nullable final CompoundLockFuture lockFut,
         @Nullable final AtomicInteger runCntr) {
+        MapQueryLazyWorker worker = MapQueryLazyWorker.currentWorker();
+
         // In presence of TX, we also must always have matching details.
         assert tx == null || txDetails != null;
 
-        assert !lazy || txDetails == null : "Lazy execution of SELECT FOR UPDATE queries is not supported.";
-
         boolean inTx = (tx != null);
 
-        MapQueryLazyWorker worker = MapQueryLazyWorker.currentWorker();
+        if (lazy && worker == null) {
+            // Lazy queries must be re-submitted to dedicated workers.
+            MapQueryLazyWorkerKey key = new MapQueryLazyWorkerKey(node.id(), reqId, segmentId);
+            worker = new MapQueryLazyWorker(ctx.igniteInstanceName(), key, log, this);
+
+            worker.submit(new Runnable() {
+                @Override public void run() {
+                    onQueryRequest0(
+                        node,
+                        reqId,
+                        segmentId,
+                        schemaName,
+                        qrys,
+                        cacheIds,
+                        topVer,
+                        partsMap,
+                        parts,
+                        pageSize,
+                        distributedJoinMode,
+                        enforceJoinOrder,
+                        replicated,
+                        timeout,
+                        params,
+                        true,
+                        mvccSnapshot,
+                        tx,
+                        txDetails,
+                        lockFut,
+                        runCntr);
+                }
+            });
+
+            if (lazyWorkerBusyLock.enterBusy()) {
+                try {
+                    MapQueryLazyWorker oldWorker = lazyWorkers.put(key, worker);
+
+                    if (oldWorker != null)
+                        oldWorker.stop(false);
 
-        if (lazy && worker == null)
-            worker = createLazyWorker(node, reqId, segmentId);
+                    IgniteThread thread = new IgniteThread(worker);
+
+                    thread.start();
+                }
+                finally {
+                    lazyWorkerBusyLock.leaveBusy();
+                }
+            }
+            else
+                log.info("Ignored query request (node is stopping) [nodeId=" + node.id() + ", reqId=" + reqId + ']');
+
+            return;
+        }
+
+        if (lazy && txDetails != null)
+            throw new IgniteSQLException("Lazy execution of SELECT FOR UPDATE queries is not supported.");
 
         // Prepare to run queries.
         GridCacheContext<?, ?> mainCctx = mainCacheContext(cacheIds);
 
         MapNodeResults nodeRess = resultsForNode(node.id());
 
-        MapQueryResults qryResults = null;
+        MapQueryResults qr = null;
 
         List<GridReservable> reserved = new ArrayList<>();
 
@@ -802,7 +847,7 @@ public class GridMapQueryExecutor {
                 if (!F.isEmpty(err)) {
                     // Unregister lazy worker because re-try may never reach this node again.
                     if (lazy)
-                        worker.stop(false);
+                        stopAndUnregisterCurrentLazyWorker();
 
                     sendRetry(node, reqId, segmentId, err);
 
@@ -810,7 +855,10 @@ public class GridMapQueryExecutor {
                 }
             }
 
-            qryResults = new MapQueryResults(h2, reqId, qrys.size(), mainCctx, worker, inTx);
+            qr = new MapQueryResults(h2, reqId, qrys.size(), mainCctx, MapQueryLazyWorker.currentWorker(), inTx);
+
+            if (nodeRess.put(reqId, segmentId, qr) != null)
+                throw new IllegalStateException();
 
             // Prepare query context.
             GridH2QueryContext qctx = new GridH2QueryContext(ctx.localNodeId(),
@@ -824,207 +872,186 @@ public class GridMapQueryExecutor {
                 .pageSize(pageSize)
                 .topologyVersion(topVer)
                 .reservations(reserved)
-                .mvccSnapshot(mvccSnapshot);
+                .mvccSnapshot(mvccSnapshot)
+                .lazyWorker(worker);
+
+            Connection conn = h2.connectionForSchema(schemaName);
+
+            H2Utils.setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder);
+
+            GridH2QueryContext.set(qctx);
 
             // qctx is set, we have to release reservations inside of it.
             reserved = null;
 
-            if (worker != null)
-                worker.queryContext(qctx);
+            try {
+                if (nodeRess.cancelled(reqId)) {
+                    GridH2QueryContext.clear(ctx.localNodeId(), node.id(), reqId, qctx.type());
 
-            GridH2QueryContext.set(qctx);
+                    nodeRess.cancelRequest(reqId);
 
-            if (nodeRess.put(reqId, segmentId, qryResults) != null)
-                throw new IllegalStateException();
+                    throw new QueryCancelledException();
+                }
 
-            Connection conn = h2.connectionForSchema(schemaName);
+                // Run queries.
+                int qryIdx = 0;
 
-            H2Utils.setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder, lazy);
+                boolean evt = mainCctx != null && mainCctx.events().isRecordable(EVT_CACHE_QUERY_EXECUTED);
 
-            if (nodeRess.cancelled(reqId)) {
-                GridH2QueryContext.clear(ctx.localNodeId(), node.id(), reqId, qctx.type());
+                for (GridCacheSqlQuery qry : qrys) {
+                    ResultSet rs = null;
 
-                nodeRess.cancelRequest(reqId);
+                    boolean removeMapping = false;
 
-                throw new QueryCancelledException();
-            }
+                    // If we are not the target node for this replicated query, just ignore it.
+                    if (qry.node() == null || (segmentId == 0 && qry.node().equals(ctx.localNodeId()))) {
+                        String sql = qry.query(); Collection<Object> params0 = F.asList(qry.parameters(params));
 
-            // Run queries.
-            int qryIdx = 0;
+                        PreparedStatement stmt;
 
-            boolean evt = mainCctx != null && mainCctx.events().isRecordable(EVT_CACHE_QUERY_EXECUTED);
+                        try {
+                            stmt = h2.prepareStatement(conn, sql, true);
+                        }
+                        catch (SQLException e) {
+                            throw new IgniteCheckedException("Failed to parse SQL query: " + sql, e);
+                        }
 
-            for (GridCacheSqlQuery qry : qrys) {
-                ResultSet rs = null;
+                        Prepared p = GridSqlQueryParser.prepared(stmt);
 
-                boolean removeMapping = false;
+                        if (GridSqlQueryParser.isForUpdateQuery(p)) {
+                            sql = GridSqlQueryParser.rewriteQueryForUpdateIfNeeded(p, inTx);
+                            stmt = h2.prepareStatement(conn, sql, true);
+                        }
 
-                // If we are not the target node for this replicated query, just ignore it.
-                if (qry.node() == null || (segmentId == 0 && qry.node().equals(ctx.localNodeId()))) {
-                    String sql = qry.query(); Collection<Object> params0 = F.asList(qry.parameters(params));
+                        h2.bindParameters(stmt, params0);
 
-                    PreparedStatement stmt;
+                        int opTimeout = IgniteH2Indexing.operationTimeout(timeout, tx);
 
-                    try {
-                        stmt = h2.prepareStatement(conn, sql, true);
-                    }
-                    catch (SQLException e) {
-                        throw new IgniteCheckedException("Failed to parse SQL query: " + sql, e);
-                    }
+                        rs = h2.executeSqlQueryWithTimer(stmt, conn, sql, params0, opTimeout, qr.queryCancel(qryIdx));
 
-                    Prepared p = GridSqlQueryParser.prepared(stmt);
+                        if (inTx) {
+                            ResultSetEnlistFuture enlistFut = ResultSetEnlistFuture.future(
+                                ctx.localNodeId(),
+                                txDetails.version(),
+                                mvccSnapshot,
+                                txDetails.threadId(),
+                                IgniteUuid.randomUuid(),
+                                txDetails.miniId(),
+                                parts,
+                                tx,
+                                opTimeout,
+                                mainCctx,
+                                rs
+                            );
 
-                    if (GridSqlQueryParser.isForUpdateQuery(p)) {
-                        sql = GridSqlQueryParser.rewriteQueryForUpdateIfNeeded(p, inTx);
-                        stmt = h2.prepareStatement(conn, sql, true);
-                    }
+                            if (lockFut != null)
+                                lockFut.register(enlistFut);
 
-                    h2.bindParameters(stmt, params0);
+                            enlistFut.init();
 
-                    int opTimeout = IgniteH2Indexing.operationTimeout(timeout, tx);
+                            enlistFut.get();
 
-                    rs = h2.executeSqlQueryWithTimer(stmt, conn, sql, params0, opTimeout, qryResults.queryCancel(qryIdx));
+                            rs.beforeFirst();
+                        }
 
-                    if (inTx) {
-                        ResultSetEnlistFuture enlistFut = ResultSetEnlistFuture.future(
-                            ctx.localNodeId(),
-                            txDetails.version(),
-                            mvccSnapshot,
-                            txDetails.threadId(),
-                            IgniteUuid.randomUuid(),
-                            txDetails.miniId(),
-                            parts,
-                            tx,
-                            opTimeout,
-                            mainCctx,
-                            rs
-                        );
-
-                        if (lockFut != null)
-                            lockFut.register(enlistFut);
-
-                        enlistFut.init();
-
-                        enlistFut.get();
-
-                        rs.beforeFirst();
-                    }
+                        if (evt) {
+                            ctx.event().record(new CacheQueryExecutedEvent<>(
+                                node,
+                                "SQL query executed.",
+                                EVT_CACHE_QUERY_EXECUTED,
+                                CacheQueryType.SQL.name(),
+                                mainCctx.name(),
+                                null,
+                                qry.query(),
+                                null,
+                                null,
+                                params,
+                                node.id(),
+                                null));
+                        }
 
-                    if (evt) {
-                        ctx.event().record(new CacheQueryExecutedEvent<>(
-                            node,
-                            "SQL query executed.",
-                            EVT_CACHE_QUERY_EXECUTED,
-                            CacheQueryType.SQL.name(),
-                            mainCctx.name(),
-                            null,
-                            qry.query(),
-                            null,
-                            null,
-                            params,
-                            node.id(),
-                            null));
+                        assert rs instanceof JdbcResultSet : rs.getClass();
                     }
 
-                    assert rs instanceof JdbcResultSet : rs.getClass();
-                }
-
-                qryResults.addResult(qryIdx, qry, node.id(), rs, params);
+                    qr.addResult(qryIdx, qry, node.id(), rs, params);
 
-                if (qryResults.cancelled()) {
-                    qryResults.result(qryIdx).close();
+                    if (qr.cancelled()) {
+                        qr.result(qryIdx).close();
 
-                    throw new QueryCancelledException();
-                }
+                        throw new QueryCancelledException();
+                    }
 
-                if (inTx) {
-                    if (tx.dht() && (runCntr == null || runCntr.decrementAndGet() == 0)) {
-                        if (removeMapping = tx.empty() && !tx.queryEnlisted())
-                            tx.rollbackAsync().get();
+                    if (inTx) {
+                        if (tx.dht() && (runCntr == null || runCntr.decrementAndGet() == 0)) {
+                            if (removeMapping = tx.empty() && !tx.queryEnlisted())
+                                tx.rollbackAsync().get();
+                        }
                     }
-                }
 
-                // Send the first page.
-                if (lockFut == null)
-                    sendNextPage(nodeRess, node, qryResults, qryIdx, segmentId, pageSize, removeMapping);
-                else {
-                    GridQueryNextPageResponse msg = prepareNextPage(nodeRess, node, qryResults, qryIdx, segmentId, pageSize, removeMapping);
-
-                    if (msg != null) {
-                        lockFut.listen(new IgniteInClosure<IgniteInternalFuture<Void>>() {
-                            @Override public void apply(IgniteInternalFuture<Void> future) {
-                                try {
-                                    if (node.isLocal())
-                                        h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg);
-                                    else
-                                        ctx.io().sendToGridTopic(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL);
-                                }
-                                catch (Exception e) {
-                                    U.error(log, e);
+                    // Send the first page.
+                    if (lockFut == null)
+                        sendNextPage(nodeRess, node, qr, qryIdx, segmentId, pageSize, removeMapping);
+                    else {
+                        GridQueryNextPageResponse msg = prepareNextPage(nodeRess, node, qr, qryIdx, segmentId, pageSize, removeMapping);
+
+                        if (msg != null) {
+                            lockFut.listen(new IgniteInClosure<IgniteInternalFuture<Void>>() {
+                                @Override public void apply(IgniteInternalFuture<Void> future) {
+                                    try {
+                                        if (node.isLocal())
+                                            h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg);
+                                        else
+                                            ctx.io().sendToGridTopic(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL);
+                                    }
+                                    catch (Exception e) {
+                                        U.error(log, e);
+                                    }
                                 }
-                            }
-                        });
+                            });
+                        }
                     }
+
+                    qryIdx++;
                 }
 
-                qryIdx++;
+                // All request results are in the memory in result set already, so it's ok to release partitions.
+                if (!lazy)
+                    releaseReservations();
             }
-
-            // All request results are in the memory in result set already, so it's ok to release partitions.
-            if (!lazy)
+            catch (Throwable e){
                 releaseReservations();
-            else if (!qryResults.isAllClosed()) {
-                if (MapQueryLazyWorker.currentWorker() == null) {
-                    final ObjectPoolReusable<H2ConnectionWrapper> detachedConn = h2.detachConnection();
 
-                    worker.start(H2Utils.session(conn), detachedConn);
-
-                    GridH2QueryContext.clearThreadLocal();
-                }
+                throw e;
             }
-            else
-                unregisterLazyWorker(worker);
         }
         catch (Throwable e) {
-            if (qryResults != null) {
-                nodeRess.remove(reqId, segmentId, qryResults);
+            if (qr != null) {
+                nodeRess.remove(reqId, segmentId, qr);
 
-                qryResults.close();
+                qr.cancel(false);
             }
-            else
-                releaseReservations();
 
-            // Stop and unregister worker after possible cancellation.
+            // Unregister worker after possible cancellation.
             if (lazy)
-                worker.stop(false);
-
-            if (e instanceof QueryCancelledException)
-                sendError(node, reqId, e);
-            else {
-                JdbcSQLException sqlEx = X.cause(e, JdbcSQLException.class);
+                stopAndUnregisterCurrentLazyWorker();
 
-                if (sqlEx != null && sqlEx.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED)
-                    sendError(node, reqId, new QueryCancelledException());
-                else {
-                    GridH2RetryException retryErr = X.cause(e, GridH2RetryException.class);
+            GridH2RetryException retryErr = X.cause(e, GridH2RetryException.class);
 
-                    if (retryErr != null) {
-                        final String retryCause = String.format(
-                            "Failed to execute non-collocated query (will retry) [localNodeId=%s, rmtNodeId=%s, reqId=%s, " +
-                                "errMsg=%s]", ctx.localNodeId(), node.id(), reqId, retryErr.getMessage()
-                        );
+            if (retryErr != null) {
+                final String retryCause = String.format(
+                    "Failed to execute non-collocated query (will retry) [localNodeId=%s, rmtNodeId=%s, reqId=%s, " +
+                    "errMsg=%s]", ctx.localNodeId(), node.id(), reqId, retryErr.getMessage()
+                );
 
-                        sendRetry(node, reqId, segmentId, retryCause);
-                    }
-                    else {
-                        U.error(log, "Failed to execute local query.", e);
+                sendRetry(node, reqId, segmentId, retryCause);
+            }
+            else {
+                U.error(log, "Failed to execute local query.", e);
 
-                        sendError(node, reqId, e);
+                sendError(node, reqId, e);
 
-                        if (e instanceof Error)
-                            throw (Error)e;
-                    }
-                }
+                if (e instanceof Error)
+                    throw (Error)e;
             }
         }
         finally {
@@ -1033,25 +1060,10 @@ public class GridMapQueryExecutor {
                 for (int i = 0; i < reserved.size(); i++)
                     reserved.get(i).release();
             }
-
-            if (MapQueryLazyWorker.currentWorker() == null && GridH2QueryContext.get() != null)
-                GridH2QueryContext.clearThreadLocal();
         }
     }
 
     /**
-     * @param node The node has sent map query request.
-     * @param reqId Request ID.
-     * @param segmentId Segment ID.
-     * @return Lazy worker.
-     */
-    private MapQueryLazyWorker createLazyWorker(ClusterNode node, long reqId, int segmentId) {
-        MapQueryLazyWorkerKey key = new MapQueryLazyWorkerKey(node.id(), reqId, segmentId);
-
-        return  new MapQueryLazyWorker(ctx.igniteInstanceName(), key, log, this);
-    }
-
-    /**
      * @param cacheIds Cache ids.
      * @return Id of the first cache in list, or {@code null} if list is empty.
      */
@@ -1076,7 +1088,6 @@ public class GridMapQueryExecutor {
     /**
      * @param node Node.
      * @param req DML request.
-     * @throws IgniteCheckedException On error.
      */
     private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) throws IgniteCheckedException {
         int[] parts = req.queryPartitions();
@@ -1244,34 +1255,24 @@ public class GridMapQueryExecutor {
             return;
         }
 
-        final MapQueryResults qryResults = nodeRess.get(req.queryRequestId(), req.segmentId());
+        final MapQueryResults qr = nodeRess.get(req.queryRequestId(), req.segmentId());
 
-        if (qryResults == null)
+        if (qr == null)
             sendError(node, req.queryRequestId(), new CacheException("No query result found for request: " + req));
-        else if (qryResults.cancelled())
+        else if (qr.cancelled())
             sendError(node, req.queryRequestId(), new QueryCancelledException());
         else {
-            MapQueryLazyWorker lazyWorker = qryResults.lazyWorker();
+            MapQueryLazyWorker lazyWorker = qr.lazyWorker();
 
             if (lazyWorker != null) {
                 lazyWorker.submit(new Runnable() {
                     @Override public void run() {
-                        try {
-                            sendNextPage(nodeRess, node, qryResults, req.query(), req.segmentId(), req.pageSize(), false);
-                        }
-                        catch (Throwable e) {
-                            JdbcSQLException sqlEx = X.cause(e, JdbcSQLException.class);
-
-                            if (sqlEx != null && sqlEx.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED)
-                                sendError(node, qryResults.queryRequestId(), new QueryCancelledException());
-                            else
-                                throw e;
-                        }
+                        sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize(), false);
                     }
                 });
             }
             else
-                sendNextPage(nodeRess, node, qryResults, req.query(), req.segmentId(), req.pageSize(), false);
+                sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize(), false);
         }
     }
 
@@ -1286,14 +1287,8 @@ public class GridMapQueryExecutor {
      * @return Next page.
      * @throws IgniteCheckedException If failed.
      */
-    private GridQueryNextPageResponse prepareNextPage(
-        MapNodeResults nodeRess,
-        ClusterNode node,
-        MapQueryResults qr,
-        int qry,
-        int segmentId,
-        int pageSize,
-        boolean removeMapping) throws IgniteCheckedException {
+    private GridQueryNextPageResponse prepareNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryResults qr, int qry, int segmentId,
+        int pageSize, boolean removeMapping) throws IgniteCheckedException {
         MapQueryResult res = qr.result(qry);
 
         assert res != null;
@@ -1314,11 +1309,8 @@ public class GridMapQueryExecutor {
                 nodeRess.remove(qr.queryRequestId(), segmentId, qr);
 
                 // Release reservations if the last page fetched, all requests are closed and this is a lazy worker.
-                if (qr.lazyWorker() != null) {
+                if (MapQueryLazyWorker.currentWorker() != null)
                     releaseReservations();
-
-                    qr.lazyWorker().stop(false);
-                }
             }
         }
 
@@ -1350,14 +1342,8 @@ public class GridMapQueryExecutor {
      * @param removeMapping Remove mapping flag.
      */
     @SuppressWarnings("unchecked")
-    private void sendNextPage(
-        MapNodeResults nodeRess,
-        ClusterNode node,
-        MapQueryResults qr,
-        int qry,
-        int segmentId,
-        int pageSize,
-        boolean removeMapping) {
+    private void sendNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryResults qr, int qry, int segmentId,
+        int pageSize, boolean removeMapping) {
         try {
             GridQueryNextPageResponse msg = prepareNextPage(nodeRess, node, qr, qry, segmentId, pageSize, removeMapping);
 
@@ -1379,7 +1365,6 @@ public class GridMapQueryExecutor {
      * @param node Node.
      * @param reqId Request ID.
      * @param segmentId Index segment ID.
-     * @param retryCause Description of the retry cause.
      */
     private void sendRetry(ClusterNode node, long reqId, int segmentId, String retryCause) {
         try {
@@ -1416,11 +1401,25 @@ public class GridMapQueryExecutor {
     }
 
     /**
+     * Unregister lazy worker if needed (i.e. if we are currently in lazy worker thread).
+     */
+    public void stopAndUnregisterCurrentLazyWorker() {
+        MapQueryLazyWorker worker = MapQueryLazyWorker.currentWorker();
+
+        if (worker != null) {
+            worker.stop(false);
+
+            // Just stop is not enough as worker may be registered, but not started due to exception.
+            unregisterLazyWorker(worker);
+        }
+    }
+
+    /**
      * Unregister lazy worker.
      *
      * @param worker Worker.
      */
-    void unregisterLazyWorker(MapQueryLazyWorker worker) {
+    public void unregisterLazyWorker(MapQueryLazyWorker worker) {
         lazyWorkers.remove(worker.key(), worker);
     }
 
@@ -1430,17 +1429,4 @@ public class GridMapQueryExecutor {
     public int registeredLazyWorkers() {
         return lazyWorkers.size();
     }
-
-    /**
-     * @param worker Worker to register.
-     */
-    void registerLazyWorker(MapQueryLazyWorker worker) {
-        MapQueryLazyWorker oldWorker = lazyWorkers.put(worker.key(), worker);
-
-        if (oldWorker != null) {
-            log.warning("Duplicates lazy worker: [key=" + worker.key() + ']');
-
-            oldWorker.stop(false);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index d9c542b..62c5c78 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -287,16 +287,11 @@ public class GridReduceQueryExecutor {
      */
     private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) {
         if (r != null) {
-            CacheException e;
+            CacheException e = new CacheException("Failed to execute map query on remote node [nodeId=" + nodeId +
+                ", errMsg=" + msg + ']');
 
-            if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR) {
-                e = new CacheException("Failed to execute map query on remote node [nodeId=" + nodeId +
-                    ", errMsg=" + msg + ']', new QueryCancelledException());
-            }
-            else {
-                e = new CacheException("Failed to execute map query on remote node [nodeId=" + nodeId +
-                    ", errMsg=" + msg + ']');
-            }
+            if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR)
+                e.addSuppressed(new QueryCancelledException());
 
             r.setStateOnException(nodeId, e);
         }
@@ -1223,9 +1218,6 @@ public class GridReduceQueryExecutor {
             }
         }
 
-        r.setStateOnException(ctx.localNodeId(),
-            new CacheException("Query is canceled.", new QueryCancelledException()));
-
         if (!runs.remove(qryReqId, r))
             U.warn(log, "Query run was already removed: " + qryReqId);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
index 217cfad..0cb986b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
@@ -17,13 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.h2.twostep;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.List;
 import java.util.NoSuchElementException;
-import java.util.RandomAccess;
 import java.util.UUID;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
@@ -73,11 +72,9 @@ public class GridResultPage {
             Collection<?> plainRows = res.plainRows();
 
             if (plainRows != null) {
-                assert plainRows instanceof RandomAccess : "instance of " + plainRows.getClass();
-
                 rowsInPage = plainRows.size();
 
-                if (rowsInPage == 0 || ((List<Value[]>)plainRows).get(0).length == res.columns())
+                if (rowsInPage == 0 || ((ArrayList<Value[]>)plainRows).get(0).length == res.columns())
                     rows = (Iterator<Value[]>)plainRows.iterator();
                 else {
                     // If it's a result of SELECT FOR UPDATE (we can tell by difference in number

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
index 8f8553a..48116d3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
@@ -17,11 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.h2.twostep;
 
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
 
 import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q;
 
@@ -85,10 +86,10 @@ class MapNodeResults {
     public void cancelRequest(long reqId) {
         for (MapRequestKey key : res.keySet()) {
             if (key.requestId() == reqId) {
-                final MapQueryResults removed = res.remove(key);
+                MapQueryResults removed = res.remove(key);
 
                 if (removed != null)
-                    removed.cancel();
+                    removed.cancel(true);
             }
         }
 
@@ -143,7 +144,7 @@ class MapNodeResults {
      */
     public void cancelAll() {
         for (MapQueryResults ress : res.values())
-            ress.cancel();
+            ress.cancel(true);
 
         // Cancel update requests
         for (GridQueryCancel upd: updCancels.values())

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
index 1cbab19..98f3df9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
@@ -20,41 +20,25 @@ package org.apache.ignite.internal.processors.query.h2.twostep;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.LongAdder;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.processors.query.h2.H2ConnectionWrapper;
-import org.apache.ignite.internal.processors.query.h2.H2Utils;
-import org.apache.ignite.internal.processors.query.h2.ObjectPoolReusable;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.thread.IgniteThread;
-import org.h2.engine.Session;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF;
-
 /**
  * Worker for lazy query execution.
  */
 public class MapQueryLazyWorker extends GridWorker {
-    /** Poll task timeout milliseconds. */
-    private static final int POLL_TASK_TIMEOUT_MS = 1000;
-
     /** Lazy thread flag. */
     private static final ThreadLocal<MapQueryLazyWorker> LAZY_WORKER = new ThreadLocal<>();
 
     /** Active lazy worker count (for testing purposes). */
     private static final LongAdder ACTIVE_CNT = new LongAdder();
 
-    /** Mutex to synchronization worker start/stop. */
-    private final Object mux = new Object();
-
     /** Task to be executed. */
     private final BlockingQueue<Runnable> tasks = new LinkedBlockingDeque<>();
 
@@ -67,14 +51,8 @@ public class MapQueryLazyWorker extends GridWorker {
     /** Latch decremented when worker finishes. */
     private final CountDownLatch stopLatch = new CountDownLatch(1);
 
-    /** Query context. */
-    private GridH2QueryContext qctx;
-
-    /** Worker is started flag. */
-    private boolean started;
-
-    /** Detached connection. */
-    private ObjectPoolReusable<H2ConnectionWrapper> detached;
+    /** Map query result. */
+    private volatile MapQueryResult res;
 
     /**
      * Constructor.
@@ -92,106 +70,38 @@ public class MapQueryLazyWorker extends GridWorker {
         this.exec = exec;
     }
 
-    /**
-     * Start lazy worker for half-processed query.
-     * In this case we have to detach H2 connection from current thread and use it for current query processing.
-     * Also tables locks must be transferred to lazy thread from QUERY_POOL thread pool.
-     *
-     * @param ses H2 Session.
-     * @param detached H2 connection detached from current thread.
-     * @throws QueryCancelledException  In case query is canceled during the worker start.
-     */
-    void start(Session ses, ObjectPoolReusable<H2ConnectionWrapper> detached) throws QueryCancelledException {
-        synchronized (mux) {
-            if (!exec.busyLock().enterBusy()) {
-                log.warning("Lazy worker isn't started. Node is stopped [key=" + key + ']');
-
-                return;
-            }
-
-            try {
-                if (started)
-                    return;
-
-                if (isCancelled) {
-                    if (detached != null)
-                        detached.recycle();
-
-                    throw new QueryCancelledException();
-                }
-
-                if (ses != null)
-                    lazyTransferStart(ses);
-
-                this.detached = detached;
-
-                exec.registerLazyWorker(this);
-
-                IgniteThread thread = new IgniteThread(this);
-
-                started = true;
-
-                thread.start();
-            }
-            finally {
-                exec.busyLock().leaveBusy();
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
         LAZY_WORKER.set(this);
 
         ACTIVE_CNT.increment();
 
-        boolean lockBusy = false;
-
         try {
-            if (qctx != null)
-                GridH2QueryContext.set(qctx);
-
-            if(detached != null)
-                lazyTransferFinish(H2Utils.session(detached.object().connection()));
-
             while (!isCancelled()) {
-                Runnable task = tasks.poll(POLL_TASK_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+                Runnable task = tasks.take();
 
                 if (task != null) {
+                    if (!exec.busyLock().enterBusy())
+                        return;
+
                     try {
                         task.run();
                     }
-                    catch (Throwable t) {
-                        log.warning("Lazy task error", t);
-                    }
-                }
-                else {
-                    try {
-                        lockBusy = false;
-
-                        if (!exec.busyLock().enterBusy()) {
-                            log.info("Stop lazy worker [key=" + key + ']');
-
-                            return;
-                        }
-
-                        lockBusy = true;
-                    }
                     finally {
-                        if (lockBusy)
-                            exec.busyLock().leaveBusy();
+                        exec.busyLock().leaveBusy();
                     }
                 }
             }
         }
         finally {
-            exec.unregisterLazyWorker(this);
+            if (res != null)
+                res.close();
 
             LAZY_WORKER.set(null);
 
             ACTIVE_CNT.decrement();
 
-            stopLatch.countDown();
+            exec.unregisterLazyWorker(this);
         }
     }
 
@@ -201,9 +111,6 @@ public class MapQueryLazyWorker extends GridWorker {
      * @param task Task to be executed.
      */
     public void submit(Runnable task) {
-        if (isCancelled)
-            return;
-
         tasks.add(task);
     }
 
@@ -218,76 +125,45 @@ public class MapQueryLazyWorker extends GridWorker {
      * Stop the worker.
      * @param nodeStop Node is stopping.
      */
-    private void stop0(boolean nodeStop) {
-        synchronized (mux) {
-            if (qctx != null && qctx.distributedJoinMode() == OFF && !qctx.isCleared())
-                qctx.clearContext(nodeStop);
+    public void stop(final boolean nodeStop) {
+        if (MapQueryLazyWorker.currentWorker() == null)
+            submit(new Runnable() {
+                @Override public void run() {
+                    stop(nodeStop);
+                }
+            });
+        else {
+            GridH2QueryContext qctx = GridH2QueryContext.get();
 
-            if (detached != null) {
-                detached.recycle();
+            if (qctx != null) {
+                qctx.clearContext(nodeStop);
 
-                detached = null;
+                GridH2QueryContext.clearThreadLocal();
             }
 
             isCancelled = true;
 
-            mux.notifyAll();
+            stopLatch.countDown();
         }
     }
 
     /**
-     * @param task Stop task.
+     * Await worker stop.
      */
-    public void submitStopTask(Runnable task) {
-        synchronized (mux) {
-            if (LAZY_WORKER.get() != null)
-                task.run();
-            else
-                submit(task);
+    public void awaitStop() {
+        try {
+            U.await(stopLatch);
         }
-    }
-
-    /**
-     * Stop the worker.
-     * @param nodeStop Node is stopping.
-     */
-    public void stop(final boolean nodeStop) {
-        synchronized (mux) {
-            if (isCancelled)
-                return;
-
-            if (started && currentWorker() == null) {
-                submit(new Runnable() {
-                    @Override public void run() {
-                        stop0(nodeStop);
-                    }
-                });
-
-                awaitStop();
-            }
-            else if (currentWorker() != null)
-                stop0(nodeStop);
+        catch (IgniteInterruptedCheckedException e) {
+            throw new IgniteException("Failed to wait for lazy worker stop (interrupted): " + name(), e);
         }
     }
 
     /**
-     * Await worker stop.
+     * @param res Map query result.
      */
-    private void awaitStop() {
-        synchronized (mux) {
-            try {
-                if (!isCancelled)
-                    mux.wait();
-
-                U.await(stopLatch);
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw new IgniteException("Failed to wait for lazy worker stop (interrupted): " + name(), e);
-            }
-            catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-            }
-        }
+    public void result(MapQueryResult res) {
+        this.res = res;
     }
 
     /**
@@ -305,13 +181,6 @@ public class MapQueryLazyWorker extends GridWorker {
     }
 
     /**
-     * @param qctx Query context.
-     */
-    public void queryContext(GridH2QueryContext qctx) {
-        this.qctx = qctx;
-    }
-
-    /**
      * Construct worker name.
      *
      * @param instanceName Instance name.
@@ -322,32 +191,4 @@ public class MapQueryLazyWorker extends GridWorker {
         return "query-lazy-worker_" + instanceName + "_" + key.nodeId() + "_" + key.queryRequestId() + "_" +
             key.segment();
     }
-
-    /**
-     * Start session transfer to lazy thread.
-     *
-     * @param ses Session.
-     */
-    private static void lazyTransferStart(Session ses) {
-        GridH2QueryContext qctx = GridH2QueryContext.get();
-
-        assert qctx != null;
-
-        for(GridH2Table tbl : qctx.lockedTables())
-            tbl.onLazyTransferStarted(ses);
-    }
-
-    /**
-     * Finish session transfer to lazy thread.
-     *
-     * @param ses Session.
-     */
-    private static void lazyTransferFinish(Session ses) {
-        GridH2QueryContext qctx = GridH2QueryContext.get();
-
-        assert qctx != null;
-
-        for(GridH2Table tbl : qctx.lockedTables())
-            tbl.onLazyTransferFinished(ses);
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
index 5a0c410..fb928c4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
@@ -22,7 +22,6 @@ import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.events.CacheQueryReadEvent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -61,9 +60,6 @@ class MapQueryResult {
         }
     }
 
-    /** Logger. */
-    private final IgniteLogger log;
-
     /** Indexing. */
     private final IgniteH2Indexing h2;
 
@@ -100,23 +96,26 @@ class MapQueryResult {
     /** */
     private final Object[] params;
 
+    /** Lazy worker. */
+    private final MapQueryLazyWorker lazyWorker;
+
     /**
-     * @param h2 H2 indexing.
      * @param rs Result set.
      * @param cctx Cache context.
      * @param qrySrcNodeId Query source node.
      * @param qry Query.
      * @param params Query params.
+     * @param lazyWorker Lazy worker.
      */
     MapQueryResult(IgniteH2Indexing h2, ResultSet rs, @Nullable GridCacheContext cctx,
-        UUID qrySrcNodeId, GridCacheSqlQuery qry, Object[] params) {
-        this.log = h2.kernalContext().log(MapQueryResult.class);
+        UUID qrySrcNodeId, GridCacheSqlQuery qry, Object[] params, @Nullable MapQueryLazyWorker lazyWorker) {
         this.h2 = h2;
         this.cctx = cctx;
         this.qry = qry;
         this.params = params;
         this.qrySrcNodeId = qrySrcNodeId;
         this.cpNeeded = F.eq(h2.kernalContext().localNodeId(), qrySrcNodeId);
+        this.lazyWorker = lazyWorker;
 
         if (rs != null) {
             this.rs = rs;
@@ -175,6 +174,8 @@ class MapQueryResult {
      * @return {@code true} If there are no more rows available.
      */
     synchronized boolean fetchNextPage(List<Value[]> rows, int pageSize) {
+        assert lazyWorker == null || lazyWorker == MapQueryLazyWorker.currentWorker();
+
         if (closed)
             return true;
 
@@ -258,13 +259,30 @@ class MapQueryResult {
      * Close the result.
      */
     public void close() {
+        if (lazyWorker != null && MapQueryLazyWorker.currentWorker() == null) {
+            lazyWorker.submit(new Runnable() {
+                @Override public void run() {
+                    close();
+                }
+            });
+
+            lazyWorker.awaitStop();
+
+            return;
+        }
+
         synchronized (this) {
+            assert lazyWorker == null || lazyWorker == MapQueryLazyWorker.currentWorker();
+
             if (closed)
                 return;
 
             closed = true;
 
-            U.close(rs, log);
+            U.closeQuiet(rs);
+
+            if (lazyWorker != null)
+                lazyWorker.stop(false);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
index b13137c..76527bc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
@@ -30,7 +30,7 @@ import org.jetbrains.annotations.Nullable;
  * Mapper query results.
  */
 class MapQueryResults {
-    /** H2 indexing. */
+    /** H@ indexing. */
     private final IgniteH2Indexing h2;
 
     /** */
@@ -113,7 +113,10 @@ class MapQueryResults {
      * @param params Query arguments.
      */
     void addResult(int qry, GridCacheSqlQuery q, UUID qrySrcNodeId, ResultSet rs, Object[] params) {
-        MapQueryResult res = new MapQueryResult(h2, rs, cctx, qrySrcNodeId, q, params);
+        MapQueryResult res = new MapQueryResult(h2, rs, cctx, qrySrcNodeId, q, params, lazyWorker);
+
+        if (lazyWorker != null)
+            lazyWorker.result(res);
 
         if (!results.compareAndSet(qry, null, res))
             throw new IllegalStateException();
@@ -136,37 +139,28 @@ class MapQueryResults {
     /**
      * Cancels the query.
      */
-    void cancel() {
+    void cancel(boolean forceQryCancel) {
         if (cancelled)
             return;
 
         cancelled = true;
 
         for (int i = 0; i < results.length(); i++) {
-            GridQueryCancel cancel = cancels[i];
+            MapQueryResult res = results.get(i);
 
-            if (cancel != null)
-                cancel.cancel();
-        }
+            if (res != null) {
+                res.close();
 
-        if (lazyWorker == null)
-            close();
-        else {
-            lazyWorker.submitStopTask(this::close);
+                continue;
+            }
 
-            lazyWorker.stop(false);
-        }
-    }
+            // NB: Cancel is already safe even for lazy queries (see implementation of passed Runnable).
+            if (forceQryCancel) {
+                GridQueryCancel cancel = cancels[i];
 
-    /**
-     *
-     */
-    public void close() {
-        for (int i = 0; i < results.length(); i++) {
-            MapQueryResult res = results.get(i);
-
-            if (res != null)
-                res.close();
+                if (cancel != null)
+                    cancel.cancel();
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLazyQueryPartitionsReleaseTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLazyQueryPartitionsReleaseTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLazyQueryPartitionsReleaseTest.java
index a991530..a112969 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLazyQueryPartitionsReleaseTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLazyQueryPartitionsReleaseTest.java
@@ -96,6 +96,7 @@ public class GridCacheLazyQueryPartitionsReleaseTest extends GridCommonAbstractT
         int partsFilled = fillAllPartitions(cache, aff);
 
         SqlFieldsQuery qry = new SqlFieldsQuery("select name, age from person")
+            .setLazy(true)
             .setPageSize(1);
 
         FieldsQueryCursor<List<?>> qryCursor = cache.query(qry);
@@ -142,6 +143,7 @@ public class GridCacheLazyQueryPartitionsReleaseTest extends GridCommonAbstractT
         int partsFilled = fillAllPartitions(cache, aff);
 
         SqlFieldsQuery qry = new SqlFieldsQuery("select name, age from person")
+            .setLazy(true)
             .setPageSize(1);
 
         FieldsQueryCursor<List<?>> qryCursor = cache.query(qry);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
index 24e2fb2..59be138 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java
@@ -121,15 +121,12 @@ public class IgniteCacheQueryH2IndexingLeakTest extends GridCommonAbstractTest {
     private static int getStatementCacheSize(GridQueryProcessor qryProcessor) {
         IgniteH2Indexing h2Idx = GridTestUtils.getFieldValue(qryProcessor, GridQueryProcessor.class, "idx");
 
-        ConcurrentMap<Thread, ConcurrentMap<H2ConnectionWrapper, Boolean>> conns =
-            GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "conns");
+        ConcurrentMap<Thread, H2ConnectionWrapper> conns = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "conns");
 
         int cntr = 0;
 
-        for (ConcurrentMap<H2ConnectionWrapper, Boolean> connPerThread: conns.values()) {
-            for (H2ConnectionWrapper w : connPerThread.keySet())
-                cntr += w.statementCacheSize();
-        }
+        for (H2ConnectionWrapper w : conns.values())
+            cntr += w.statementCacheSize();
 
         return cntr;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
index 67a9501..56fd7b8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
@@ -100,83 +100,84 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
 
     /** */
     public void testRemoteQueryExecutionTimeout() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableTimeout() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel0() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel1() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel2() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel3() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel0() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel1() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel2() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel3() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel0() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel1() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel2() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel3() throws Exception {
-        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryAlreadyFinishedStop() throws Exception {
-        testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, false);
+        testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit,
-                                 boolean timeout, boolean checkCanceled) throws Exception {
+                                 boolean timeout) throws Exception {
         try (Ignite client = startGrid("client")) {
+
             IgniteCache<Object, Object> cache = client.cache(DEFAULT_CACHE_NAME);
 
             assertEquals(0, cache.localSize());
@@ -203,8 +204,7 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
                 qry.setTimeout(timeoutUnits, timeUnit);
 
                 cursor = cache.query(qry);
-            }
-            else {
+            } else {
                 cursor = cache.query(qry);
 
                 client.scheduler().runLocal(new Runnable() {
@@ -214,11 +214,8 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
                 }, timeoutUnits, timeUnit);
             }
 
-            try (QueryCursor<List<?>> ignored = cursor) {
-                cursor.getAll();
-
-                if (checkCanceled)
-                    fail("Query not canceled");
+            try(QueryCursor<List<?>> ignored = cursor) {
+                cursor.iterator();
             }
             catch (CacheException ex) {
                 log().error("Got expected exception", ex);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
index 48b43a7..7e23c88 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
@@ -67,11 +67,6 @@ public class IgniteCacheQueryAbstractDistributedJoinSelfTest extends GridCommonA
         "where pr.companyId = co._key\n" +
         "order by co._key, pr._key ";
 
-    protected static final String QRY_LONG = "select pe.id, co.id, pr._key\n" +
-        "from \"pe\".Person pe, \"pr\".Product pr, \"co\".Company co, \"pu\".Purchase pu\n" +
-        "where pe._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" +
-        "order by pe.id desc";
-
     /** */
     protected static final int GRID_CNT = 2;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
index 3beebff..bad5303 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
@@ -33,7 +33,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerArray;
-import org.apache.ignite.testframework.GridTestUtils;
 
 /**
  * Test for distributed queries with node restarts.
@@ -102,11 +101,11 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends IgniteCa
 
         assertEquals(broadcastQry, plan.contains("batched:broadcast"));
 
-        final List<List<?>> goldenRes = grid(0).cache("pu").query(qry0).getAll();
+        final List<List<?>> pRes = grid(0).cache("pu").query(qry0).getAll();
 
         Thread.sleep(3000);
 
-        assertEquals(goldenRes, grid(0).cache("pu").query(qry0).getAll());
+        assertEquals(pRes, grid(0).cache("pu").query(qry0).getAll());
 
         final SqlFieldsQuery qry1;
 
@@ -123,7 +122,7 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends IgniteCa
 
         final List<List<?>> rRes = grid(0).cache("co").query(qry1).getAll();
 
-        assertFalse(goldenRes.isEmpty());
+        assertFalse(pRes.isEmpty());
         assertFalse(rRes.isEmpty());
 
         final AtomicInteger qryCnt = new AtomicInteger();
@@ -162,12 +161,9 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends IgniteCa
                             qry.setPageSize(smallPageSize ? 30 : 1000);
 
                             try {
-                                assertEquals(goldenRes, cache.query(qry).getAll());
+                                assertEquals(pRes, cache.query(qry).getAll());
                             }
                             catch (CacheException e) {
-                                if (!smallPageSize)
-                                    log.error("Unexpected exception at the test", e);
-
                                 assertTrue("On large page size must retry.", smallPageSize);
 
                                 boolean failedOnRemoteFetch = false;
@@ -267,7 +263,7 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends IgniteCa
             }
         }, restartThreadsNum, "restart-thread");
 
-        GridTestUtils.waitForCondition(() -> fail.get(), duration);
+        Thread.sleep(duration);
 
         info("Stopping...");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
index 9f8a2fc..03a8d49 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
@@ -40,47 +40,47 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest extends IgniteCacheQueryAbstractDistributedJoinSelfTest {
     /** */
     public void testCancel1() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 1, TimeUnit.MILLISECONDS, false, true);
+        testQueryCancel(grid(0), "pe", QRY_0, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testCancel2() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 50, TimeUnit.MILLISECONDS, false, true);
+        testQueryCancel(grid(0), "pe", QRY_0, 50, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testCancel3() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 100, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(grid(0), "pe", QRY_0, 100, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testCancel4() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 500, TimeUnit.MILLISECONDS, false, false);
+        testQueryCancel(grid(0), "pe", QRY_0, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testTimeout1() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 1, TimeUnit.MILLISECONDS, true, true);
+        testQueryCancel(grid(0), "pe", QRY_0, 1, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testTimeout2() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 50, TimeUnit.MILLISECONDS, true, true);
+        testQueryCancel(grid(0), "pe", QRY_0, 50, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testTimeout3() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 100, TimeUnit.MILLISECONDS, true, false);
+        testQueryCancel(grid(0), "pe", QRY_0, 100, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testTimeout4() throws Exception {
-        testQueryCancel(grid(0), "pe", QRY_LONG, 500, TimeUnit.MILLISECONDS, true, false);
+        testQueryCancel(grid(0), "pe", QRY_0, 500, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     private void testQueryCancel(Ignite ignite, String cacheName, String sql, int timeoutUnits, TimeUnit timeUnit,
-                           boolean timeout, boolean checkCanceled) throws Exception {
+                           boolean timeout) throws Exception {
         SqlFieldsQuery qry = new SqlFieldsQuery(sql).setDistributedJoins(true);
 
         IgniteCache<Object, Object> cache = ignite.cache(cacheName);
@@ -101,10 +101,7 @@ public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest extend
         }
 
         try (QueryCursor<List<?>> ignored = cursor) {
-            cursor.getAll();
-
-            if (checkCanceled)
-                fail("Query not canceled");
+            cursor.iterator();
         }
         catch (CacheException ex) {
             log().error("Got expected exception", ex);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
index 4d02b2e..072f1ab 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
@@ -627,8 +627,6 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      * @throws Exception If failed.
      */
     public void testQueryConsistencyMultithreaded() throws Exception {
-        final int KEY_COUNT = 5000;
-
         // Start complex topology.
         ignitionStart(serverConfiguration(1));
         ignitionStart(serverConfiguration(2));
@@ -640,7 +638,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         run(cli, createSql);
 
-        put(cli, 0, KEY_COUNT);
+        put(cli, 0, 5000);
 
         final AtomicBoolean stopped = new AtomicBoolean();
 
@@ -698,7 +696,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                     List<Cache.Entry<BinaryObject, BinaryObject>> res = cache.query(
                         new SqlQuery<BinaryObject, BinaryObject>(valTypeName, "from " + TBL_NAME)).getAll();
 
-                    assertEquals(KEY_COUNT, res.size());
+                    assertEquals(5000, res.size());
                 }
 
                 return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
index fe45ed6..7713004 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java
@@ -160,7 +160,7 @@ public class H2ConnectionLeaksSelfTest extends GridCommonAbstractTest {
                 Map<Thread, ?> conns = perThreadConnections(i);
 
                 for(Thread t : conns.keySet())
-                    log.error("Connection is not closed for thread: " + t.getName());
+                    log.error("+++ Connection is not closed for thread: " + t.getName());
             }
 
             fail("H2 JDBC connections leak detected. See the log above.");


[21/50] [abbrv] ignite git commit: IGNITE-5935: MVCC TX: Tx recovery protocol

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/PartitionCountersNeighborcastFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/PartitionCountersNeighborcastFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/PartitionCountersNeighborcastFuture.java
new file mode 100644
index 0000000..f7fe9cb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/PartitionCountersNeighborcastFuture.java
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.transactions;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheCompoundIdentityFuture;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastRequest;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
+
+/**
+ * Represents partition update counters delivery to remote nodes.
+ */
+public class PartitionCountersNeighborcastFuture extends GridCacheCompoundIdentityFuture<Void> {
+    /** */
+    private final IgniteUuid futId = IgniteUuid.randomUuid();
+    /** */
+    private boolean trackable = true;
+    /** */
+    private final GridCacheSharedContext<?, ?> cctx;
+    /** */
+    private final IgniteInternalTx tx;
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    public PartitionCountersNeighborcastFuture(
+        IgniteInternalTx tx, GridCacheSharedContext<?, ?> cctx) {
+        super(null);
+
+        this.tx = tx;
+
+        this.cctx = cctx;
+
+        log = cctx.logger(CU.TX_MSG_RECOVERY_LOG_CATEGORY);
+    }
+
+    /**
+     * Starts processing.
+     */
+    public void init() {
+        if (log.isInfoEnabled()) {
+            log.info("Starting delivery partition countres to remote nodes [txId=" + tx.nearXidVersion() +
+                ", futId=" + futId);
+        }
+
+        HashSet<UUID> siblings = siblingBackups();
+
+        cctx.mvcc().addFuture(this, futId);
+
+        for (UUID peer : siblings) {
+            List<PartitionUpdateCountersMessage> cntrs = cctx.tm().txHandler()
+                .filterUpdateCountersForBackupNode(tx, cctx.node(peer));
+
+            if (F.isEmpty(cntrs))
+                continue;
+
+            MiniFuture miniFut = new MiniFuture(peer);
+
+            try {
+                cctx.io().send(peer, new PartitionCountersNeighborcastRequest(cntrs, futId), SYSTEM_POOL);
+
+                add(miniFut);
+            }
+            catch (IgniteCheckedException e) {
+                if (!(e instanceof ClusterTopologyCheckedException))
+                    log.warning("Failed to send partition counters to remote node [node=" + peer + ']', e);
+                else
+                    logNodeLeft(peer);
+
+                miniFut.onDone();
+            }
+        }
+
+        markInitialized();
+    }
+
+    /** */
+    private HashSet<UUID> siblingBackups() {
+        Map<UUID, Collection<UUID>> txNodes = tx.transactionNodes();
+
+        assert txNodes != null;
+
+        UUID locNodeId = cctx.localNodeId();
+
+        HashSet<UUID> siblings = new HashSet<>();
+
+        txNodes.values().stream()
+            .filter(backups -> backups.contains(locNodeId))
+            .forEach(siblings::addAll);
+
+        siblings.remove(locNodeId);
+
+        return siblings;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) {
+        boolean comp = super.onDone(res, err);
+
+        if (comp)
+            cctx.mvcc().removeFuture(futId);
+
+        return comp;
+    }
+
+    /**
+     * Processes a response from a remote peer. Completes a mini future for that peer.
+     *
+     * @param nodeId Remote peer node id.
+     */
+    public void onResult(UUID nodeId) {
+        if (log.isInfoEnabled())
+            log.info("Remote peer acked partition counters delivery [futId=" + futId +
+                ", node=" + nodeId + ']');
+
+        completeMini(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onNodeLeft(UUID nodeId) {
+        logNodeLeft(nodeId);
+
+        // if a left node is one of remote peers then a mini future for it is completed successfully
+        completeMini(nodeId);
+
+        return true;
+    }
+
+    /** */
+    private void completeMini(UUID nodeId) {
+        for (IgniteInternalFuture<?> fut : futures()) {
+            assert fut instanceof MiniFuture;
+
+            MiniFuture mini = (MiniFuture)fut;
+
+            if (mini.nodeId.equals(nodeId)) {
+                cctx.kernalContext().closure().runLocalSafe(mini::onDone);
+
+                break;
+            }
+        }
+    }
+
+    /** */
+    private void logNodeLeft(UUID nodeId) {
+        if (log.isInfoEnabled()) {
+            log.info("Failed during partition counters delivery to remote node. " +
+                "Node left cluster (will ignore) [futId=" + futId +
+                ", node=" + nodeId + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid futureId() {
+        return futId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean trackable() {
+        return trackable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void markNotTrackable() {
+        trackable = false;
+    }
+
+    /**
+     * Component of compound parent future. Represents interaction with one of remote peers.
+     */
+    private static class MiniFuture extends GridFutureAdapter<Void> {
+        /** */
+        private final UUID nodeId;
+
+        /** */
+        private MiniFuture(UUID nodeId) {
+            this.nodeId = nodeId;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
index e1a0bd6..550ec09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Values which should be tracked during transaction execution and applied on commit.
@@ -69,7 +70,7 @@ public class TxCounters {
     /**
      * @return Final update counters.
      */
-    public Collection<PartitionUpdateCountersMessage> updateCounters() {
+    @Nullable public Collection<PartitionUpdateCountersMessage> updateCounters() {
         return updCntrs;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
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 4569f65..fbfd99b 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
@@ -1423,7 +1423,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                 if (cache != null && !cache.isLocal() && cache.context().userCache())
                     req.addUpdateCounters(ctx.localNodeId(),
-                        toCountersMap(cache.context().topology().localUpdateCounters(false)));
+                        toCountersMap(cache.context().topology().localUpdateCounters(false, false)));
             }
         }
 
@@ -1564,7 +1564,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                                 if (cache != null && !cache.isLocal() && cache.context().userCache()) {
                                     CachePartitionPartialCountersMap cntrsMap =
-                                        cache.context().topology().localUpdateCounters(false);
+                                        cache.context().topology().localUpdateCounters(false, false);
 
                                     cntrs = U.marshal(marsh, cntrsMap);
                                 }
@@ -2504,7 +2504,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                         if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
                             cntrsPerNode.put(ctx.localNodeId(),
-                                toCountersMap(cctx.topology().localUpdateCounters(false)));
+                                toCountersMap(cctx.topology().localUpdateCounters(false, false)));
 
                         routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
index 8bdfafe..a7880a2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
@@ -314,4 +314,4 @@ public abstract class IgniteTxOriginatingNodeFailureAbstractSelfTest extends Gri
     private boolean ignoredMessage(GridIoMessage msg) {
         return ignoreMsgCls != null && ignoreMsgCls.isAssignableFrom(msg.message().getClass());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
index 7514555..3f55e9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
@@ -23,10 +23,12 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.StreamSupport;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
@@ -40,6 +42,8 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
@@ -58,6 +62,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 
 /**
@@ -268,8 +273,10 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
 
                         assertNotNull(cache);
 
-                        assertEquals("Failed to check entry value on node: " + checkNodeId,
-                            fullFailure ? initVal : val, cache.localPeek(key));
+                        if (atomicityMode() != TRANSACTIONAL_SNAPSHOT) {
+                            assertEquals("Failed to check entry value on node: " + checkNodeId,
+                                fullFailure ? initVal : val, cache.localPeek(key));
+                        }
 
                         return null;
                     }
@@ -278,8 +285,22 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
         }
 
         for (Map.Entry<Integer, String> e : map.entrySet()) {
-            for (Ignite g : G.allGrids())
-                assertEquals(fullFailure ? initVal : e.getValue(), g.cache(DEFAULT_CACHE_NAME).get(e.getKey()));
+            long cntr0 = -1;
+
+            for (Ignite g : G.allGrids()) {
+                Integer key = e.getKey();
+
+                assertEquals(fullFailure ? initVal : e.getValue(), g.cache(DEFAULT_CACHE_NAME).get(key));
+
+                if (g.affinity(DEFAULT_CACHE_NAME).isPrimaryOrBackup(((IgniteEx)g).localNode(), key)) {
+                    long nodeCntr = updateCoutner(g, key);
+
+                    if (cntr0 == -1)
+                        cntr0 = nodeCntr;
+
+                    assertEquals(cntr0, nodeCntr);
+                }
+            }
         }
     }
 
@@ -402,6 +423,9 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
 
             assertFalse(e.getValue().isEmpty());
 
+            if (atomicityMode() == TRANSACTIONAL_SNAPSHOT)
+                continue;
+
             for (ClusterNode node : e.getValue()) {
                 final UUID checkNodeId = node.id();
 
@@ -425,8 +449,22 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
         }
 
         for (Map.Entry<Integer, String> e : map.entrySet()) {
-            for (Ignite g : G.allGrids())
-                assertEquals(!commmit ? initVal : e.getValue(), g.cache(DEFAULT_CACHE_NAME).get(e.getKey()));
+            long cntr0 = -1;
+
+            for (Ignite g : G.allGrids()) {
+                Integer key = e.getKey();
+
+                assertEquals(!commmit ? initVal : e.getValue(), g.cache(DEFAULT_CACHE_NAME).get(key));
+
+                if (g.affinity(DEFAULT_CACHE_NAME).isPrimaryOrBackup(((IgniteEx)g).localNode(), key)) {
+                    long nodeCntr = updateCoutner(g, key);
+
+                    if (cntr0 == -1)
+                        cntr0 = nodeCntr;
+
+                    assertEquals(cntr0, nodeCntr);
+                }
+            }
         }
     }
 
@@ -529,4 +567,21 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
         else
             return false;
     }
-}
\ No newline at end of file
+
+    /** */
+    private static long updateCoutner(Ignite ign, Object key) {
+        return dataStore(((IgniteEx)ign).cachex(DEFAULT_CACHE_NAME).context(), key)
+            .map(IgniteCacheOffheapManager.CacheDataStore::updateCounter)
+            .orElse(0L);
+    }
+
+    /** */
+    private static Optional<IgniteCacheOffheapManager.CacheDataStore> dataStore(
+        GridCacheContext<?, ?> cctx, Object key) {
+        int p = cctx.affinity().partition(key);
+
+        return StreamSupport.stream(cctx.offheap().cacheDataStores().spliterator(), false)
+            .filter(ds -> ds.partId() == p)
+            .findFirst();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
index 07bbf6c..81d4796 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
@@ -148,4 +148,4 @@ public class GridCachePartitionedTxOriginatingNodeFailureSelfTest extends
 
         testTxOriginatingNodeFails(keys, false);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.java
index 23304a4..bb3fff0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.java
@@ -34,4 +34,4 @@ public class IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest
 
         return ccfg;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePrimaryNodeFailureRecoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePrimaryNodeFailureRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePrimaryNodeFailureRecoveryAbstractTest.java
index 00f9729..b0d083d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePrimaryNodeFailureRecoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePrimaryNodeFailureRecoveryAbstractTest.java
@@ -19,8 +19,11 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
+import java.util.stream.StreamSupport;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
@@ -33,10 +36,13 @@ import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
@@ -55,6 +61,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.internal.processors.cache.ExchangeContext.IGNITE_EXCHANGE_COMPATIBILITY_VER_1;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
@@ -114,6 +121,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryNodeFailureRecovery1() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryNodeFailure(false, false, true);
     }
 
@@ -121,6 +130,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryNodeFailureRecovery2() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryNodeFailure(true, false, true);
     }
 
@@ -128,6 +139,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryNodeFailureRollback1() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryNodeFailure(false, true, true);
     }
 
@@ -135,8 +148,11 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryNodeFailureRollback2() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryNodeFailure(true, true, true);
     }
+
     /**
      * @throws Exception If failed.
      */
@@ -245,8 +261,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
         GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 try {
-                    checkKey(key1, rollback ? null : key1Nodes);
-                    checkKey(key2, rollback ? null : key2Nodes);
+                    checkKey(key1, rollback, key1Nodes, 0);
+                    checkKey(key2, rollback, key2Nodes, 0);
 
                     return true;
                 }
@@ -258,14 +274,16 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
             }
         }, 5000);
 
-        checkKey(key1, rollback ? null : key1Nodes);
-        checkKey(key2, rollback ? null : key2Nodes);
+        checkKey(key1, rollback, key1Nodes, 0);
+        checkKey(key2, rollback, key2Nodes, 0);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryAndOriginatingNodeFailureRecovery1() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryAndOriginatingNodeFailure(false, false, true);
     }
 
@@ -273,6 +291,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryAndOriginatingNodeFailureRecovery2() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryAndOriginatingNodeFailure(true, false, true);
     }
 
@@ -280,6 +300,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryAndOriginatingNodeFailureRollback1() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryAndOriginatingNodeFailure(false, true, true);
     }
 
@@ -287,6 +309,8 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
      * @throws Exception If failed.
      */
     public void testOptimisticPrimaryAndOriginatingNodeFailureRollback2() throws Exception {
+        if (atomicityMode() == TRANSACTIONAL_SNAPSHOT) return;
+
         primaryAndOriginatingNodeFailure(true, true, true);
     }
 
@@ -327,14 +351,14 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
     private void primaryAndOriginatingNodeFailure(final boolean locBackupKey,
         final boolean rollback,
         boolean optimistic)
-        throws Exception
-    {
+        throws Exception {
         // TODO IGNITE-6174: when exchanges can be merged test fails because of IGNITE-6174.
         System.setProperty(IGNITE_EXCHANGE_COMPATIBILITY_VER_1, "true");
 
         try {
-            IgniteCache<Integer, Integer> cache0 = jcache(0);
-            IgniteCache<Integer, Integer> cache2 = jcache(2);
+            int orig = 0;
+
+            IgniteCache<Integer, Integer> origCache = jcache(orig);
 
             Affinity<Integer> aff = ignite(0).affinity(DEFAULT_CACHE_NAME);
 
@@ -342,7 +366,7 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
 
             for (int key = 0; key < 10_000; key++) {
                 if (aff.isPrimary(ignite(1).cluster().localNode(), key)) {
-                    if (locBackupKey == aff.isBackup(ignite(0).cluster().localNode(), key)) {
+                    if (locBackupKey == aff.isBackup(ignite(orig).cluster().localNode(), key)) {
                         key0 = key;
 
                         break;
@@ -353,27 +377,27 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
             assertNotNull(key0);
 
             final Integer key1 = key0;
-            final Integer key2 = primaryKey(cache2);
+            final Integer key2 = primaryKey(jcache(2));
 
-            int backups = cache0.getConfiguration(CacheConfiguration.class).getBackups();
+            int backups = origCache.getConfiguration(CacheConfiguration.class).getBackups();
 
             final Collection<ClusterNode> key1Nodes =
-                (locBackupKey && backups < 2) ? null : aff.mapKeyToPrimaryAndBackups(key1);
+                (locBackupKey && backups < 2) ? Collections.emptyList() : aff.mapKeyToPrimaryAndBackups(key1);
             final Collection<ClusterNode> key2Nodes = aff.mapKeyToPrimaryAndBackups(key2);
 
-            TestCommunicationSpi commSpi = (TestCommunicationSpi)ignite(0).configuration().getCommunicationSpi();
+            TestCommunicationSpi commSpi = (TestCommunicationSpi)ignite(orig).configuration().getCommunicationSpi();
 
-            IgniteTransactions txs = ignite(0).transactions();
+            IgniteTransactions txs = ignite(orig).transactions();
 
             Transaction tx = txs.txStart(optimistic ? OPTIMISTIC : PESSIMISTIC, REPEATABLE_READ);
 
             log.info("Put key1 [key1=" + key1 + ", nodes=" + U.nodeIds(aff.mapKeyToPrimaryAndBackups(key1)) + ']');
 
-            cache0.put(key1, key1);
+            origCache.put(key1, key1);
 
             log.info("Put key2 [key2=" + key2 + ", nodes=" + U.nodeIds(aff.mapKeyToPrimaryAndBackups(key2)) + ']');
 
-            cache0.put(key2, key2);
+            origCache.put(key2, key2);
 
             log.info("Start prepare.");
 
@@ -399,13 +423,13 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
 
             log.info("Stop originating node.");
 
-            stopGrid(0);
+            stopGrid(orig);
 
             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                 @Override public boolean apply() {
                     try {
-                        checkKey(key1, rollback ? null : key1Nodes);
-                        checkKey(key2, rollback ? null : key2Nodes);
+                        checkKey(key1, rollback, key1Nodes, 0);
+                        checkKey(key2, rollback, key2Nodes, 0);
 
                         return true;
                     } catch (AssertionError e) {
@@ -416,24 +440,23 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
                 }
             }, 5000);
 
-            checkKey(key1, rollback ? null : key1Nodes);
-            checkKey(key2, rollback ? null : key2Nodes);
+            checkKey(key1, rollback, key1Nodes, 0);
+            checkKey(key2, rollback, key2Nodes, 0);
         }
         finally {
             System.clearProperty(IGNITE_EXCHANGE_COMPATIBILITY_VER_1);
         }
     }
 
-    /**
-     * @param key Key.
-     * @param keyNodes Key nodes.
-     */
-    private void checkKey(Integer key, Collection<ClusterNode> keyNodes) {
-        if (keyNodes == null) {
-            for (Ignite ignite : G.allGrids()) {
-                IgniteCache<Integer, Integer> cache = ignite.cache(DEFAULT_CACHE_NAME);
+    /** */
+    private void checkKey(Integer key, boolean rollback, Collection<ClusterNode> keyNodes, long initUpdCntr) {
+        if (rollback) {
+            if (atomicityMode() != TRANSACTIONAL_SNAPSHOT) {
+                for (Ignite ignite : G.allGrids()) {
+                    IgniteCache<Integer, Integer> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
-                assertNull("Unexpected value for: " + ignite.name(), cache.localPeek(key));
+                    assertNull("Unexpected value for: " + ignite.name(), cache.localPeek(key));
+                }
             }
 
             for (Ignite ignite : G.allGrids()) {
@@ -441,10 +464,34 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
 
                 assertNull("Unexpected value for: " + ignite.name(), cache.get(key));
             }
+
+            boolean found = keyNodes.isEmpty();
+
+            long cntr0 = -1;
+
+            for (ClusterNode node : keyNodes) {
+                try {
+                    long nodeCntr = updateCoutner(grid(node), key);
+
+                    found = true;
+
+                    if (cntr0 == -1)
+                        cntr0 = nodeCntr;
+
+                    assertEquals(cntr0, nodeCntr);
+                }
+                catch (IgniteIllegalStateException ignore) {
+                    // No-op.
+                }
+            }
+
+            assertTrue("Failed to find key node.", found);
         }
-        else {
+        else if (!keyNodes.isEmpty()) {
             boolean found = false;
 
+            long cntr0 = -1;
+
             for (ClusterNode node : keyNodes) {
                 try {
                     Ignite ignite = grid(node);
@@ -454,6 +501,13 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
                     ignite.cache(DEFAULT_CACHE_NAME);
 
                     assertEquals("Unexpected value for: " + ignite.name(), key, key);
+
+                    long nodeCntr = updateCoutner(ignite, key);
+
+                    if (cntr0 == -1)
+                        cntr0 = nodeCntr;
+
+                    assertTrue(nodeCntr == cntr0 && nodeCntr > initUpdCntr);
                 }
                 catch (IgniteIllegalStateException ignore) {
                     // No-op.
@@ -498,6 +552,23 @@ public abstract class IgniteCachePrimaryNodeFailureRecoveryAbstractTest extends
         assertTrue("Failed to wait for tx.", wait);
     }
 
+    /** */
+    private static long updateCoutner(Ignite ign, Object key) {
+        return dataStore(((IgniteEx)ign).cachex(DEFAULT_CACHE_NAME).context(), key)
+            .map(IgniteCacheOffheapManager.CacheDataStore::updateCounter)
+            .orElse(0L);
+    }
+
+    /** */
+    private static Optional<IgniteCacheOffheapManager.CacheDataStore> dataStore(
+        GridCacheContext<?, ?> cctx, Object key) {
+        int p = cctx.affinity().partition(key);
+
+        return StreamSupport.stream(cctx.offheap().cacheDataStores().spliterator(), false)
+            .filter(ds -> ds.partId() == p)
+            .findFirst();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxOriginatingNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxOriginatingNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxOriginatingNodeFailureSelfTest.java
index 79308c8..8730c5c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxOriginatingNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxOriginatingNodeFailureSelfTest.java
@@ -35,4 +35,4 @@ public class GridCacheReplicatedTxOriginatingNodeFailureSelfTest extends
     @Override protected Class<?> ignoreMessageClass() {
         return GridDistributedTxPrepareRequest.class;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
index 35da7a4..ca3c09f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -74,6 +73,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestQueryC
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTx;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.lang.GridInClosure3;
 import org.apache.ignite.internal.util.typedef.CI1;
@@ -3181,7 +3181,7 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
         MvccProcessorImpl crd = mvccProcessor(node);
 
         // Start query to prevent cleanup.
-        IgniteInternalFuture<MvccSnapshot> fut = crd.requestSnapshotAsync();
+        IgniteInternalFuture<MvccSnapshot> fut = crd.requestSnapshotAsync((IgniteInternalTx)null);
 
         fut.get();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 91c702e..0fef7b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -521,7 +521,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
             Affinity<Object> aff = grid(i).affinity(DEFAULT_CACHE_NAME);
 
             CachePartitionPartialCountersMap act = grid(i).cachex(DEFAULT_CACHE_NAME).context().topology()
-                .localUpdateCounters(false);
+                .localUpdateCounters(false, false);
 
             for (Map.Entry<Integer, Long> e : updCntrs.entrySet()) {
                 if (aff.mapPartitionToPrimaryAndBackups(e.getKey()).contains(grid(i).localNode())) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
index a2c6c83..0cdd0c4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
@@ -61,4 +61,4 @@ public class IgniteCacheTxRecoverySelfTestSuite extends TestSuite {
 
         return suite;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTxRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTxRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTxRecoveryTest.java
new file mode 100644
index 0000000..01f50cc
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTxRecoveryTest.java
@@ -0,0 +1,654 @@
+/*
+ * 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.cache.mvcc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.IntPredicate;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.Transaction;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxRecoveryTest.NodeMode.CLIENT;
+import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxRecoveryTest.NodeMode.SERVER;
+import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxRecoveryTest.TxEndResult.COMMIT;
+import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxRecoveryTest.TxEndResult.ROLLBAK;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionState.COMMITTED;
+import static org.apache.ignite.transactions.TransactionState.PREPARED;
+import static org.apache.ignite.transactions.TransactionState.PREPARING;
+import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK;
+
+/** */
+public class CacheMvccTxRecoveryTest extends CacheMvccAbstractTest {
+    /** */
+    public enum TxEndResult {
+        /** */ COMMIT,
+        /** */ ROLLBAK
+    }
+
+    /** */
+    public enum NodeMode {
+        /** */ SERVER,
+        /** */ CLIENT
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        throw new RuntimeException("Is not supposed to be used");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryCommitNearFailure1() throws Exception {
+        checkRecoveryNearFailure(COMMIT, CLIENT);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryCommitNearFailure2() throws Exception {
+        checkRecoveryNearFailure(COMMIT, SERVER);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryRollbackNearFailure1() throws Exception {
+        checkRecoveryNearFailure(ROLLBAK, CLIENT);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryRollbackNearFailure2() throws Exception {
+        checkRecoveryNearFailure(ROLLBAK, SERVER);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryCommitPrimaryFailure1() throws Exception {
+        checkRecoveryPrimaryFailure(COMMIT, false);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryRollbackPrimaryFailure1() throws Exception {
+        checkRecoveryPrimaryFailure(ROLLBAK, false);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryCommitPrimaryFailure2() throws Exception {
+        checkRecoveryPrimaryFailure(COMMIT, true);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryRollbackPrimaryFailure2() throws Exception {
+        checkRecoveryPrimaryFailure(ROLLBAK, true);
+    }
+
+    /** */
+    private void checkRecoveryNearFailure(TxEndResult endRes, NodeMode nearNodeMode) throws Exception {
+        int gridCnt = 4;
+        int baseCnt = gridCnt - 1;
+
+        boolean commit = endRes == COMMIT;
+
+        startGridsMultiThreaded(baseCnt);
+
+        // tweak client/server near
+        client = nearNodeMode == CLIENT;
+
+        IgniteEx nearNode = startGrid(baseCnt);
+
+        IgniteCache<Object, Object> cache = nearNode.getOrCreateCache(basicCcfg()
+            .setBackups(1));
+
+        Affinity<Object> aff = nearNode.affinity(DEFAULT_CACHE_NAME);
+
+        List<Integer> keys = new ArrayList<>();
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(grid(0).localNode(), i) && aff.isBackup(grid(1).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(grid(1).localNode(), i) && aff.isBackup(grid(2).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        assert keys.size() == 2;
+
+        TestRecordingCommunicationSpi nearComm
+            = (TestRecordingCommunicationSpi)nearNode.configuration().getCommunicationSpi();
+
+        if (!commit)
+            nearComm.blockMessages(GridNearTxPrepareRequest.class, grid(1).name());
+
+        GridTestUtils.runAsync(() -> {
+            // run in separate thread to exclude tx from thread-local map
+            GridNearTxLocal nearTx
+                = ((TransactionProxyImpl)nearNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)).tx();
+
+            for (Integer k : keys)
+                cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(k));
+
+            List<IgniteInternalTx> txs = IntStream.range(0, baseCnt)
+                .mapToObj(i -> txsOnNode(grid(i), nearTx.xidVersion()))
+                .flatMap(Collection::stream)
+                .collect(Collectors.toList());
+
+            IgniteInternalFuture<?> prepareFut = nearTx.prepareNearTxLocal();
+
+            if (commit)
+                prepareFut.get();
+            else
+                assertConditionEventually(() -> txs.stream().anyMatch(tx -> tx.state() == PREPARED));
+
+            // drop near
+            nearNode.close();
+
+            assertConditionEventually(() -> txs.stream().allMatch(tx -> tx.state() == (commit ? COMMITTED : ROLLED_BACK)));
+
+            return null;
+        }).get();
+
+        if (commit) {
+            assertConditionEventually(() -> {
+                int rowsCnt = grid(0).cache(DEFAULT_CACHE_NAME)
+                    .query(new SqlFieldsQuery("select * from Integer")).getAll().size();
+                return rowsCnt == keys.size();
+            });
+        }
+        else {
+            int rowsCnt = G.allGrids().get(0).cache(DEFAULT_CACHE_NAME)
+                .query(new SqlFieldsQuery("select * from Integer")).getAll().size();
+
+            assertEquals(0, rowsCnt);
+        }
+
+        assertPartitionCountersAreConsistent(keys, grids(baseCnt, i -> true));
+    }
+
+    /** */
+    private void checkRecoveryPrimaryFailure(TxEndResult endRes, boolean mvccCrd) throws Exception {
+        int gridCnt = 4;
+        int baseCnt = gridCnt - 1;
+
+        boolean commit = endRes == COMMIT;
+
+        startGridsMultiThreaded(baseCnt);
+
+        client = true;
+
+        IgniteEx nearNode = startGrid(baseCnt);
+
+        IgniteCache<Object, Object> cache = nearNode.getOrCreateCache(basicCcfg()
+            .setBackups(1));
+
+        Affinity<Object> aff = nearNode.affinity(DEFAULT_CACHE_NAME);
+
+        List<Integer> keys = new ArrayList<>();
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(grid(0).localNode(), i) && aff.isBackup(grid(1).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(grid(1).localNode(), i) && aff.isBackup(grid(2).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        assert keys.size() == 2;
+
+        int victim, victimBackup;
+
+        if (mvccCrd) {
+            victim = 0;
+            victimBackup = 1;
+        }
+        else {
+            victim = 1;
+            victimBackup = 2;
+        }
+
+        TestRecordingCommunicationSpi victimComm = (TestRecordingCommunicationSpi)grid(victim).configuration().getCommunicationSpi();
+
+        if (commit)
+            victimComm.blockMessages(GridNearTxFinishResponse.class, nearNode.name());
+        else
+            victimComm.blockMessages(GridDhtTxPrepareRequest.class, grid(victimBackup).name());
+
+        GridNearTxLocal nearTx
+            = ((TransactionProxyImpl)nearNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)).tx();
+
+        for (Integer k : keys)
+            cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(k));
+
+        List<IgniteInternalTx> txs = IntStream.range(0, baseCnt)
+            .filter(i -> i != victim)
+            .mapToObj(i -> txsOnNode(grid(i), nearTx.xidVersion()))
+            .flatMap(Collection::stream)
+            .collect(Collectors.toList());
+
+        IgniteInternalFuture<IgniteInternalTx> commitFut = nearTx.commitAsync();
+
+        if (commit)
+            assertConditionEventually(() -> txs.stream().allMatch(tx -> tx.state() == COMMITTED));
+        else
+            assertConditionEventually(() -> txs.stream().anyMatch(tx -> tx.state() == PREPARED));
+
+        // drop victim
+        grid(victim).close();
+
+        awaitPartitionMapExchange();
+
+        assertConditionEventually(() -> txs.stream().allMatch(tx -> tx.state() == (commit ? COMMITTED : ROLLED_BACK)));
+
+        assert victimComm.hasBlockedMessages();
+
+        if (commit) {
+            assertConditionEventually(() -> {
+                int rowsCnt = G.allGrids().get(0).cache(DEFAULT_CACHE_NAME)
+                    .query(new SqlFieldsQuery("select * from Integer")).getAll().size();
+                return rowsCnt == keys.size();
+            });
+        }
+        else {
+            int rowsCnt = G.allGrids().get(0).cache(DEFAULT_CACHE_NAME)
+                .query(new SqlFieldsQuery("select * from Integer")).getAll().size();
+
+            assertEquals(0, rowsCnt);
+        }
+
+        assertTrue(commitFut.isDone());
+
+        assertPartitionCountersAreConsistent(keys, grids(baseCnt, i -> i != victim));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRecoveryCommit() throws Exception {
+        startGridsMultiThreaded(2);
+
+        client = true;
+
+        IgniteEx ign = startGrid(2);
+
+        IgniteCache<Object, Object> cache = ign.getOrCreateCache(basicCcfg());
+
+        AtomicInteger keyCntr = new AtomicInteger();
+
+        ArrayList<Integer> keys = new ArrayList<>();
+
+        ign.cluster().forServers().nodes()
+            .forEach(node -> keys.add(keyForNode(ign.affinity(DEFAULT_CACHE_NAME), keyCntr, node)));
+
+        GridTestUtils.runAsync(() -> {
+            // run in separate thread to exclude tx from thread-local map
+            Transaction tx = ign.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            for (Integer k : keys)
+                cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(k));
+
+            ((TransactionProxyImpl)tx).tx().prepareNearTxLocal().get();
+
+            return null;
+        }).get();
+
+        // drop near
+        stopGrid(2, true);
+
+        IgniteEx srvNode = grid(0);
+
+        assertConditionEventually(
+            () -> srvNode.cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("select * from Integer")).getAll().size() == 2
+        );
+
+        assertPartitionCountersAreConsistent(keys, G.allGrids());
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testCountersNeighborcastServerFailed() throws Exception {
+        int srvCnt = 4;
+
+        startGridsMultiThreaded(srvCnt);
+
+        client = true;
+
+        IgniteEx ign = startGrid(srvCnt);
+
+        IgniteCache<Object, Object> cache = ign.getOrCreateCache(basicCcfg()
+            .setBackups(2));
+
+        ArrayList<Integer> keys = new ArrayList<>();
+
+        int vid = 3;
+
+        IgniteEx victim = grid(vid);
+
+        Affinity<Object> aff = ign.affinity(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(victim.localNode(), i) && !aff.isBackup(grid(0).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        for (int i = 0; i < 100; i++) {
+            if (aff.isPrimary(victim.localNode(), i) && !aff.isBackup(grid(1).localNode(), i)) {
+                keys.add(i);
+                break;
+            }
+        }
+
+        assert keys.size() == 2 && !keys.contains(99);
+
+        // prevent prepare on one backup
+        ((TestRecordingCommunicationSpi)victim.configuration().getCommunicationSpi())
+            .blockMessages(GridDhtTxPrepareRequest.class, grid(0).name());
+
+        GridNearTxLocal nearTx = ((TransactionProxyImpl)ign.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)).tx();
+
+        for (Integer k : keys)
+            cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(k));
+
+        List<IgniteInternalTx> txs = IntStream.range(0, srvCnt)
+            .mapToObj(this::grid)
+            .filter(g -> g != victim)
+            .map(g -> txsOnNode(g, nearTx.xidVersion()))
+            .flatMap(Collection::stream)
+            .collect(Collectors.toList());
+
+        nearTx.commitAsync();
+
+        // await tx partially prepared
+        assertConditionEventually(() -> txs.stream().anyMatch(tx -> tx.state() == PREPARED));
+
+        CountDownLatch latch1 = new CountDownLatch(1);
+        CountDownLatch latch2 = new CountDownLatch(1);
+
+        IgniteInternalFuture<Object> backgroundTxFut = GridTestUtils.runAsync(() -> {
+            try (Transaction ignored = ign.transactions().txStart()) {
+                boolean upd = false;
+
+                for (int i = 100; i < 200; i++) {
+                    if (!aff.isPrimary(victim.localNode(), i)) {
+                        cache.put(i, 11);
+                        upd = true;
+                        break;
+                    }
+                }
+
+                assert upd;
+
+                latch1.countDown();
+
+                latch2.await();
+            }
+
+            return null;
+        });
+
+        latch1.await();
+
+        // drop primary
+        victim.close();
+
+        // do all assertions before rebalance
+        assertConditionEventually(() -> txs.stream().allMatch(tx -> tx.state() == ROLLED_BACK));
+
+        List<IgniteEx> liveNodes = grids(srvCnt, i -> i != vid);
+
+        assertPartitionCountersAreConsistent(keys, liveNodes);
+
+        latch2.countDown();
+
+        backgroundTxFut.get();
+
+        assertTrue(liveNodes.stream()
+            .map(node -> node.cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("select * from Integer")).getAll())
+            .allMatch(Collection::isEmpty));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testUpdateCountersGapIsClosed() throws Exception {
+        int srvCnt = 3;
+
+        startGridsMultiThreaded(srvCnt);
+
+        client = true;
+
+        IgniteEx ign = startGrid(srvCnt);
+
+        IgniteCache<Object, Object> cache = ign.getOrCreateCache(
+            basicCcfg().setBackups(2));
+
+        int vid = 1;
+
+        IgniteEx victim = grid(vid);
+
+        ArrayList<Integer> keys = new ArrayList<>();
+
+        Integer part = null;
+
+        Affinity<Object> aff = ign.affinity(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < 2000; i++) {
+            int p = aff.partition(i);
+            if (aff.isPrimary(victim.localNode(), i)) {
+                if (part == null) part = p;
+                if (p == part) keys.add(i);
+                if (keys.size() == 2) break;
+            }
+        }
+
+        assert keys.size() == 2;
+
+        Transaction txA = ign.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+        // prevent first transaction prepare on backups
+        ((TestRecordingCommunicationSpi)victim.configuration().getCommunicationSpi())
+            .blockMessages(new IgniteBiPredicate<ClusterNode, Message>() {
+                final AtomicInteger limiter = new AtomicInteger();
+
+                @Override public boolean apply(ClusterNode node, Message msg) {
+                    if (msg instanceof GridDhtTxPrepareRequest)
+                        return limiter.getAndIncrement() < 2;
+
+                    return false;
+                }
+            });
+
+        cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(keys.get(0)));
+
+        txA.commitAsync();
+
+        GridCacheVersion aXidVer = ((TransactionProxyImpl)txA).tx().xidVersion();
+
+        assertConditionEventually(() -> txsOnNode(victim, aXidVer).stream()
+            .anyMatch(tx -> tx.state() == PREPARING));
+
+        GridTestUtils.runAsync(() -> {
+            try (Transaction txB = ign.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, 42)").setArgs(keys.get(1)));
+
+                txB.commit();
+            }
+        }).get();
+
+        long victimUpdCntr = updateCounter(victim.cachex(DEFAULT_CACHE_NAME).context(), keys.get(0));
+
+        List<IgniteEx> backupNodes = grids(srvCnt, i -> i != vid);
+
+        List<IgniteInternalTx> backupTxsA = backupNodes.stream()
+            .map(node -> txsOnNode(node, aXidVer))
+            .flatMap(Collection::stream)
+            .collect(Collectors.toList());
+
+        // drop primary
+        victim.close();
+
+        assertConditionEventually(() -> backupTxsA.stream().allMatch(tx -> tx.state() == ROLLED_BACK));
+
+        backupNodes.stream()
+            .map(node -> node.cache(DEFAULT_CACHE_NAME))
+            .forEach(c -> {
+                assertEquals(1, c.query(new SqlFieldsQuery("select * from Integer")).getAll().size());
+            });
+
+        backupNodes.forEach(node -> {
+            for (Integer k : keys)
+                assertEquals(victimUpdCntr, updateCounter(node.cachex(DEFAULT_CACHE_NAME).context(), k));
+        });
+    }
+
+    /** */
+    private static CacheConfiguration<Object, Object> basicCcfg() {
+        return new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Integer.class);
+    }
+
+    /** */
+    private static List<IgniteInternalTx> txsOnNode(IgniteEx node, GridCacheVersion xidVer) {
+        List<IgniteInternalTx> txs = node.context().cache().context().tm().activeTransactions().stream()
+            .peek(tx -> assertEquals(xidVer, tx.nearXidVersion()))
+            .collect(Collectors.toList());
+
+        assert !txs.isEmpty();
+
+        return txs;
+    }
+
+    /** */
+    private static void assertConditionEventually(GridAbsPredicate p)
+        throws IgniteInterruptedCheckedException {
+        if (!GridTestUtils.waitForCondition(p, 5_000))
+            fail();
+    }
+
+    /** */
+    private List<IgniteEx> grids(int cnt, IntPredicate p) {
+        return IntStream.range(0, cnt).filter(p).mapToObj(this::grid).collect(Collectors.toList());
+    }
+
+    /** */
+    private void assertPartitionCountersAreConsistent(Iterable<Integer> keys, Iterable<? extends Ignite> nodes) {
+        for (Integer key : keys) {
+            long cntr0 = -1;
+
+            for (Ignite n : nodes) {
+                IgniteEx node = ((IgniteEx)n);
+
+                if (node.affinity(DEFAULT_CACHE_NAME).isPrimaryOrBackup(node.localNode(), key)) {
+                    long cntr = updateCounter(node.cachex(DEFAULT_CACHE_NAME).context(), key);
+//                    System.err.println(node.localNode().consistentId() + " " + key + " -> " + cntr);
+                    if (cntr0 == -1)
+                        cntr0 = cntr;
+
+                    assertEquals(cntr0, cntr);
+                }
+            }
+        }
+    }
+
+    /** */
+    private static long updateCounter(GridCacheContext<?, ?> cctx, Object key) {
+        return dataStore(cctx, key)
+            .map(IgniteCacheOffheapManager.CacheDataStore::updateCounter)
+            .get();
+    }
+
+    /** */
+    private static Optional<IgniteCacheOffheapManager.CacheDataStore> dataStore(
+        GridCacheContext<?, ?> cctx, Object key) {
+        int p = cctx.affinity().partition(key);
+        IgniteCacheOffheapManager offheap = cctx.offheap();
+        return StreamSupport.stream(offheap.cacheDataStores().spliterator(), false)
+            .filter(ds -> ds.partId() == p)
+            .findFirst();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
index cf68546..a0d492c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -84,7 +85,7 @@ public class GridIndexRebuildWithMvccEnabledSelfTest extends GridIndexRebuildSel
      * @throws IgniteCheckedException if failed.
      */
     private static void lockVersion(IgniteEx node) throws IgniteCheckedException {
-        node.context().coordinators().requestSnapshotAsync().get();
+        node.context().coordinators().requestSnapshotAsync((IgniteInternalTx)null).get();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
index ce2a130..15045c9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
@@ -18,6 +18,12 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxPessimisticOriginatingNodeFailureSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxPessimisticOriginatingNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.index.MvccEmptyTransactionSelfTest;
 import org.apache.ignite.internal.processors.cache.index.SqlTransactionsCommandsWithMvccEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccBasicContinuousQueryTest;
@@ -60,10 +66,13 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlLockTimeoutT
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlUpdateCountersTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccStreamingInsertTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxNodeMappingTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxRecoveryTest;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccRepeatableReadBulkOpsTest;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccRepeatableReadOperationsTest;
 import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildWithMvccEnabledSelfTest;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+
 /**
  *
  */
@@ -140,6 +149,55 @@ public class IgniteCacheMvccSqlTestSuite extends TestSuite {
         suite.addTestSuite(CacheMvccContinuousWithTransformerPartitionedSelfTest.class);
         suite.addTestSuite(CacheMvccContinuousWithTransformerReplicatedSelfTest.class);
 
+        // Transaction recovery.
+        suite.addTestSuite(CacheMvccTxRecoveryTest.class);
+
+        suite.addTestSuite(MvccPartitionedPrimaryNodeFailureRecoveryTest.class);
+        suite.addTestSuite(MvccPartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.class);
+        suite.addTestSuite(MvccColocatedTxPessimisticOriginatingNodeFailureRecoveryTest.class);
+        suite.addTestSuite(MvccReplicatedTxPessimisticOriginatingNodeFailureRecoveryTest.class);
+
         return suite;
     }
+
+    /** */
+    public static class MvccPartitionedPrimaryNodeFailureRecoveryTest
+        extends IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest {
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL_SNAPSHOT;
+        }
+    }
+
+    /** */
+    public static class MvccPartitionedTwoBackupsPrimaryNodeFailureRecoveryTest
+        extends IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest {
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL_SNAPSHOT;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected NearCacheConfiguration nearConfiguration() {
+            return null;
+        }
+    }
+
+    /** */
+    public static class MvccColocatedTxPessimisticOriginatingNodeFailureRecoveryTest
+        extends GridCacheColocatedTxPessimisticOriginatingNodeFailureSelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL_SNAPSHOT;
+        }
+    }
+
+    /** */
+    public static class MvccReplicatedTxPessimisticOriginatingNodeFailureRecoveryTest
+        extends GridCacheReplicatedTxPessimisticOriginatingNodeFailureSelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL_SNAPSHOT;
+        }
+    }
 }


[37/50] [abbrv] ignite git commit: IGNITE-9960: SQL: Reverted IGNITE-9171 and IGNITE-9864 until performance is fixed. This closes #5045.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
index 140eb6e..d5cc0eb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
@@ -17,21 +17,13 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker;
@@ -39,11 +31,16 @@ import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
 /**
  * Tests for lazy query execution.
  */
 public class LazyQuerySelfTest extends GridCommonAbstractTest {
-    /** Keys count. */
+    /** Keys ocunt. */
     private static final int KEY_CNT = 200;
 
     /** Base query argument. */
@@ -97,91 +94,6 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test DDL operation on table with high load queries.
-     *
-     * @throws Exception If failed.
-     */
-    public void testTableWriteLockStarvation() throws Exception {
-        final Ignite srv = startGrid(1);
-
-        srv.createCache(cacheConfiguration(4));
-
-        populateBaseQueryData(srv);
-
-        final AtomicBoolean end = new AtomicBoolean(false);
-
-        final int qryThreads = 10;
-
-        final CountDownLatch latch = new CountDownLatch(qryThreads);
-
-        // Do many concurrent queries.
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                latch.countDown();
-
-                while(!end.get()) {
-                    FieldsQueryCursor<List<?>> cursor = execute(srv, query(0)
-                        .setPageSize(PAGE_SIZE_SMALL));
-
-                    cursor.getAll();
-                }
-            }
-        }, qryThreads, "usr-qry");
-
-        latch.await();
-
-        Thread.sleep(500);
-
-        execute(srv, new SqlFieldsQuery("CREATE INDEX PERSON_NAME ON Person (name asc)")).getAll();
-        execute(srv, new SqlFieldsQuery("DROP INDEX PERSON_NAME")).getAll();
-
-        // Test is OK in case DDL operations is passed on hi load queries pressure.
-        end.set(true);
-        fut.get();
-    }
-
-    /**
-     * Test release reserved partition after query complete (results is bigger than one page).
-     *
-     * @throws Exception If failed.
-     */
-    public void testReleasePartitionReservationSeveralPagesResults() throws Exception {
-        checkReleasePartitionReservation(PAGE_SIZE_SMALL);
-    }
-
-    /**
-     * Test release reserved partition after query complete (results is placed on one page).
-     *
-     * @throws Exception If failed.
-     */
-    public void testReleasePartitionReservationOnePageResults() throws Exception {
-        checkReleasePartitionReservation(KEY_CNT);
-    }
-
-    /**
-     * Test release reserved partition after query complete.
-     *
-     * @param pageSize Results page size.
-     * @throws Exception If failed.
-     */
-    public void checkReleasePartitionReservation(int pageSize) throws Exception {
-        Ignite srv1 = startGrid(1);
-        startGrid(2);
-
-        srv1.createCache(cacheConfiguration(1));
-
-        populateBaseQueryData(srv1);
-
-        FieldsQueryCursor<List<?>> cursor = execute(srv1, query(0).setPageSize(pageSize));
-
-        cursor.getAll();
-
-        startGrid(3);
-
-        awaitPartitionMapExchange();
-    }
-
-    /**
      * Check local query execution.
      *
      * @param parallelism Query parallelism.
@@ -239,18 +151,18 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
         assertNoWorkers();
 
         // Test server node leave with active worker.
-        FieldsQueryCursor<List<?>> cursor2 = execute(srv1, baseQuery().setPageSize(PAGE_SIZE_SMALL));
+        cursor = execute(srv1, baseQuery().setPageSize(PAGE_SIZE_SMALL));
 
         try {
-            Iterator<List<?>> iter2 = cursor2.iterator();
+            iter = cursor.iterator();
 
             for (int i = 0; i < 30; i++)
-                iter2.next();
+                iter.next();
 
             stopGrid(2);
         }
         finally {
-            cursor2.close();
+            cursor.close();
         }
 
         assertNoWorkers();
@@ -321,55 +233,7 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
             }
         }
 
-        checkHoldLazyQuery(node);
-
-        checkShortLazyQuery(node);
-    }
-
-    /**
-     * @param node Ignite node.
-     * @throws Exception If failed.
-     */
-    public void checkHoldLazyQuery(Ignite node) throws Exception {
-        ArrayList rows = new ArrayList<>();
-
-        FieldsQueryCursor<List<?>> cursor0 = execute(node, query(BASE_QRY_ARG).setPageSize(PAGE_SIZE_SMALL));
-
-        // Do many concurrent queries to Test full iteration.
-        GridTestUtils.runMultiThreaded(new Runnable() {
-            @Override public void run() {
-                for (int i = 0; i < 5; ++i) {
-                    FieldsQueryCursor<List<?>> cursor = execute(node, query(KEY_CNT - PAGE_SIZE_SMALL + 1)
-                        .setPageSize(PAGE_SIZE_SMALL));
-
-                    cursor.getAll();
-                }
-            }
-        }, 5, "usr-qry");
-
-        for (List<?> row : cursor0)
-            rows.add(row);
-
-        assertBaseQueryResults(rows);
-    }
-
-    /**
-     * @param node Ignite node.
-     * @throws Exception If failed.
-     */
-    public void checkShortLazyQuery(Ignite node) throws Exception {
-        ArrayList rows = new ArrayList<>();
-
-        FieldsQueryCursor<List<?>> cursor0 = execute(node, query(KEY_CNT - PAGE_SIZE_SMALL + 1).setPageSize(PAGE_SIZE_SMALL));
-
-        Iterator<List<?>> it = cursor0.iterator();
-
         assertNoWorkers();
-
-        while (it.hasNext())
-            rows.add(it.next());
-
-        assertQueryResults(rows, KEY_CNT - PAGE_SIZE_SMALL + 1);
     }
 
     /**
@@ -403,11 +267,8 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
      * @return Default cache configuration.
      */
     private static CacheConfiguration<Long, Person> cacheConfiguration(int parallelism) {
-        return new CacheConfiguration<Long, Person>()
-            .setName(CACHE_NAME)
-            .setIndexedTypes(Long.class, Person.class)
-            .setQueryParallelism(parallelism)
-            .setAffinity(new RendezvousAffinityFunction(false, 10));
+        return new CacheConfiguration<Long, Person>().setName(CACHE_NAME).setIndexedTypes(Long.class, Person.class)
+            .setQueryParallelism(parallelism);
     }
 
     /**
@@ -417,7 +278,7 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
      * @return Query.
      */
     private static SqlFieldsQuery query(long arg) {
-        return new SqlFieldsQuery("SELECT id, name FROM Person WHERE id >= " + arg);
+        return new SqlFieldsQuery("SELECT id, name FROM Person WHERE id >= ?").setArgs(arg);
     }
 
     /**
@@ -426,23 +287,13 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
      * @param rows Result rows.
      */
     private static void assertBaseQueryResults(List<List<?>> rows) {
-        assertQueryResults(rows, BASE_QRY_ARG);
-    }
-
-    /**
-     * Assert base query results.
-     *
-     * @param rows Result rows.
-     * @param resSize Result size.
-     */
-    private static void assertQueryResults(List<List<?>> rows, int resSize) {
-        assertEquals(KEY_CNT - resSize, rows.size());
+        assertEquals(KEY_CNT - BASE_QRY_ARG, rows.size());
 
         for (List<?> row : rows) {
             Long id = (Long)row.get(0);
             String name = (String)row.get(1);
 
-            assertTrue(id >= resSize);
+            assertTrue(id >= BASE_QRY_ARG);
             assertEquals(nameForId(id), name);
         }
     }
@@ -466,7 +317,7 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings("unchecked")
     private static FieldsQueryCursor<List<?>> execute(Ignite node, SqlFieldsQuery qry) {
-        return cache(node).query(qry);
+        return cache(node).query(qry.setLazy(true));
     }
 
     /**
@@ -474,8 +325,8 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
-    private void assertNoWorkers() throws Exception {
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+    private static void assertNoWorkers() throws Exception {
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 for (Ignite node : Ignition.allGrids()) {
                     IgniteH2Indexing idx = (IgniteH2Indexing) ((IgniteKernal)node).context().query().getIndexing();
@@ -486,22 +337,7 @@ public class LazyQuerySelfTest extends GridCommonAbstractTest {
 
                 return MapQueryLazyWorker.activeCount() == 0;
             }
-        }, 1000L)) {
-            log.error("Lazy workers on nodes:");
-
-            for (Ignite node : Ignition.allGrids()) {
-                IgniteH2Indexing idx = (IgniteH2Indexing) ((IgniteKernal)node).context().query().getIndexing();
-
-                if (idx.mapQueryExecutor().registeredLazyWorkers() != 0) {
-                    log.error("[node=" + node + ", " + "registeredLazyWorkers="
-                        + idx.mapQueryExecutor().registeredLazyWorkers() + ']');
-                }
-
-                log.error("Active lazy workers: " + MapQueryLazyWorker.activeCount());
-
-                fail("There are not stopped lazy workers. See error message above.");
-            }
-        }
+        }, 1000L);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
deleted file mode 100644
index 9be0870..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.query.h2;
-
-import java.util.concurrent.CompletableFuture;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class ObjectPoolSelfTest extends GridCommonAbstractTest {
-    /** */
-    private ObjectPool<Obj> pool = new ObjectPool<>(Obj::new, 1, null, null);
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectIsReusedAfterRecycling() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-
-        Obj o1 = r1.object();
-
-        r1.recycle();
-
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        Obj o2 = r2.object();
-
-        assertSame(o1, o2);
-
-        assertFalse(o1.isClosed());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBorrowedObjectIsNotReturnedTwice() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        assertNotSame(r1.object(), r2.object());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectShouldBeClosedOnRecycleIfPoolIsFull() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        Obj o2 = r2.object();
-
-        r1.recycle();
-        r2.recycle();
-
-        assertNull(r1.object());
-        assertNull(r2.object());
-
-        assertTrue(o2.isClosed());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectShouldNotBeReturnedIfPoolIsFull() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        r1.recycle();
-
-        assertEquals(1, pool.bagSize());
-
-        r2.recycle();
-
-        assertEquals(1, pool.bagSize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectShouldReturnedToBag() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-
-        CompletableFuture.runAsync(() -> {
-            r1.recycle();
-
-            assertEquals(1, pool.bagSize());
-        }).join();
-
-        assertEquals(1, pool.bagSize());
-    }
-
-    /** */
-    private static class Obj implements AutoCloseable {
-        /** */
-        private boolean closed = false;
-
-        /** {@inheritDoc} */
-        @Override public void close() {
-            closed = true;
-        }
-
-        /**
-         * @return {@code True} if closed.
-         */
-        public boolean isClosed() {
-            return closed;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
new file mode 100644
index 0000000..b7b7a37
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.query.h2;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPool.Reusable;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class ThreadLocalObjectPoolSelfTest extends GridCommonAbstractTest {
+    /** */
+    private ThreadLocalObjectPool<Obj> pool = new ThreadLocalObjectPool<>(Obj::new, 1);
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectIsReusedAfterRecycling() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        o1.recycle();
+        Reusable<Obj> o2 = pool.borrow();
+
+        assertSame(o1.object(), o2.object());
+        assertFalse(o1.object().isClosed());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBorrowedObjectIsNotReturnedTwice() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        Reusable<Obj> o2 = pool.borrow();
+
+        assertNotSame(o1.object(), o2.object());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectShouldBeClosedOnRecycleIfPoolIsFull() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        Reusable<Obj> o2 = pool.borrow();
+        o1.recycle();
+        o2.recycle();
+
+        assertTrue(o2.object().isClosed());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectShouldNotBeReturnedIfPoolIsFull() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        Reusable<Obj> o2 = pool.borrow();
+
+        o1.recycle();
+
+        assertEquals(1, pool.bagSize());
+
+        o2.recycle();
+
+        assertEquals(1, pool.bagSize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectShouldReturnedToRecyclingThreadBag() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+
+        CompletableFuture.runAsync(() -> {
+            o1.recycle();
+
+            assertEquals(1, pool.bagSize());
+        }).join();
+
+        assertEquals(0, pool.bagSize());
+    }
+
+    /** */
+    private static class Obj implements AutoCloseable {
+        /** */
+        private boolean closed = false;
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            closed = true;
+        }
+
+        /**
+         * @return {@code True} if closed.
+         */
+        public boolean isClosed() {
+            return closed;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
index ac467d5..dbb2c59 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
@@ -384,6 +384,11 @@ public class RetryCauseMessageSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void cancelLazyWorkers() {
+            startedExecutor.cancelLazyWorkers();
+        }
+
+        /** {@inheritDoc} */
         @Override GridSpinBusyLock busyLock() {
             return startedExecutor.busyLock();
         }
@@ -394,8 +399,19 @@ public class RetryCauseMessageSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void stopAndUnregisterCurrentLazyWorker() {
+            startedExecutor.stopAndUnregisterCurrentLazyWorker();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void unregisterLazyWorker(MapQueryLazyWorker worker) {
+            startedExecutor.unregisterLazyWorker(worker);
+        }
+
+        /** {@inheritDoc} */
         @Override public int registeredLazyWorkers() {
             return startedExecutor.registeredLazyWorkers();
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 24ff297..7633d2a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -180,8 +180,8 @@ import org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullify
 import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest;
-import org.apache.ignite.internal.processors.query.h2.ObjectPoolSelfTest;
 import org.apache.ignite.internal.processors.query.h2.PreparedStatementExSelfTest;
+import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest;
 import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
@@ -475,7 +475,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(SqlUserCommandSelfTest.class);
         suite.addTestSuite(EncryptedSqlTableTest.class);
 
-        suite.addTestSuite(ObjectPoolSelfTest.class);
+        suite.addTestSuite(ThreadLocalObjectPoolSelfTest.class);
         suite.addTestSuite(H2StatementCacheSelfTest.class);
         suite.addTestSuite(PreparedStatementExSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
index 4e4614c..519b2ed 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
@@ -54,7 +54,7 @@ namespace ignite
                     loc(false),
                     distributedJoins(false),
                     enforceJoinOrder(false),
-                    lazy(true),
+                    lazy(false),
                     args()
                 {
                     // No-op.
@@ -73,7 +73,7 @@ namespace ignite
                     loc(false),
                     distributedJoins(false),
                     enforceJoinOrder(false),
-                    lazy(true),
+                    lazy(false),
                     args()
                 {
                     // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
index c7014f9..4cfc940 100644
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@ -46,7 +46,7 @@ namespace
     const bool testEnforceJoinOrder = true;
     const bool testReplicatedOnly = true;
     const bool testCollocated = true;
-    const bool testLazy = false;
+    const bool testLazy = true;
     const bool testSkipReducerOnUpdate = true;
 
     const std::string testAddressStr = testServerHost + ':' + ignite::common::LexicalCast<std::string>(testServerPort);
@@ -234,7 +234,7 @@ void CheckDsnConfig(const Configuration& cfg)
     BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), false);
     BOOST_CHECK_EQUAL(cfg.IsReplicatedOnly(), false);
     BOOST_CHECK_EQUAL(cfg.IsCollocated(), false);
-    BOOST_CHECK_EQUAL(cfg.IsLazy(), true);
+    BOOST_CHECK_EQUAL(cfg.IsLazy(), false);
     BOOST_CHECK_EQUAL(cfg.IsSkipReducerOnUpdate(), false);
     BOOST_CHECK(cfg.GetAddresses().empty());
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/cpp/odbc/src/config/configuration.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp
index 6caf86e..a99894d 100644
--- a/modules/platforms/cpp/odbc/src/config/configuration.cpp
+++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp
@@ -43,7 +43,7 @@ namespace ignite
             const bool Configuration::DefaultValue::enforceJoinOrder = false;
             const bool Configuration::DefaultValue::replicatedOnly = false;
             const bool Configuration::DefaultValue::collocated = false;
-            const bool Configuration::DefaultValue::lazy = true;
+            const bool Configuration::DefaultValue::lazy = false;
             const bool Configuration::DefaultValue::skipReducerOnUpdate = false;
 
             const ProtocolVersion& Configuration::DefaultValue::protocolVersion = ProtocolVersion::GetCurrent();

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
index f9c318f..ceeeb37 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -761,11 +761,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         public void TestSqlQueryTimeout()
         {
             var cache = Cache();
-            PopulateCache(cache, false, 30000, x => true);
+            PopulateCache(cache, false, 20000, x => true);
 
-            var sqlQry = new SqlQuery(typeof(QueryPerson), "WHERE age < 2000")
+            var sqlQry = new SqlQuery(typeof(QueryPerson), "WHERE age < 500 AND name like '%1%'")
             {
-                Timeout = TimeSpan.FromMilliseconds(1)
+                Timeout = TimeSpan.FromMilliseconds(2)
             };
 
             // ReSharper disable once ReturnValueOfPureMethodIsNotUsed

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
index 67bde2e..f5b5baa 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
@@ -103,8 +103,6 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
 #pragma warning restore 618 // Type or member is obsolete
 
             fq = fieldsQuery.GetFieldsQuery();
-	        fq.Lazy = false;
-
             Assert.AreEqual(GetSqlEscapeAll()
                     ? "select _T0.\"Name\" from PERSON_ORG_SCHEMA.\"Person\" as _T0"
                     : "select _T0.NAME from PERSON_ORG_SCHEMA.Person as _T0",

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
index 760a48d..02d13f6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
@@ -150,9 +150,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         {
             var cache = GetClientCache<Person>();
 
-            cache.PutAll(Enumerable.Range(1, 1000).ToDictionary(x => x, x => new Person(x)));
+            cache.PutAll(Enumerable.Range(1, 30000).ToDictionary(x => x, x => new Person(x)));
 
-            var qry = new SqlFieldsQuery("select * from Person p0, Person p1, Person p2")
+            var qry = new SqlFieldsQuery("select * from Person where Name like '%ers%'")
             {
                 Timeout = TimeSpan.FromMilliseconds(1)
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
index f81e4ac..a93e00d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
@@ -36,23 +36,22 @@ namespace Apache.Ignite.Core.Cache.Query
         /// <param name="args">Arguments.</param>
         public SqlFieldsQuery(string sql, params object[] args) : this(sql, false, args)
         {
-	        Lazy = true;
+            // No-op.
         }
 
-		/// <summary>
-		/// Constructor,
-		/// </summary>
-		/// <param name="sql">SQL.</param>
-		/// <param name="loc">Whether query should be executed locally.</param>
-		/// <param name="args">Arguments.</param>
-		public SqlFieldsQuery(string sql, bool loc, params object[] args)
+        /// <summary>
+        /// Constructor,
+        /// </summary>
+        /// <param name="sql">SQL.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlFieldsQuery(string sql, bool loc, params object[] args)
         {
             Sql = sql;
             Local = loc;
             Arguments = args;
 
             PageSize = DefaultPageSize;
-	        Lazy = true;
         }
 
         /// <summary>
@@ -136,20 +135,18 @@ namespace Apache.Ignite.Core.Cache.Query
         /// </summary>
         public string Schema { get; set; }
 
-		/// <summary>
-		/// Gets or sets a value indicating whether this <see cref="SqlFieldsQuery"/> is lazy.
-		/// <para />
-		/// When lazy mode is turned off Ignite attempts to fetch the whole query result set to memory and send it to the client.
-		/// For small and medium result sets this provides optimal performance and minimize duration of internal
-		/// database locks, thus increasing concurrency.
-		/// <para />
-		/// If result set is too big to fit in available memory this could lead to excessive GC pauses and even
-		/// OutOfMemoryError. Use this flag as a hint for Ignite to fetch result set lazily, thus minimizing memory
-		/// consumption at the cost of moderate performance hit.
-		/// <para />
-		/// Default mode is lazy since version 2.7. The default value of the flag is changed to 'true'.
-		/// </summary>
-		public bool Lazy { get; set; }
+        /// <summary>
+        /// Gets or sets a value indicating whether this <see cref="SqlFieldsQuery"/> is lazy.
+        /// <para />
+        /// By default Ignite attempts to fetch the whole query result set to memory and send it to the client.
+        /// For small and medium result sets this provides optimal performance and minimize duration of internal
+        /// database locks, thus increasing concurrency.
+        /// <para />
+        /// If result set is too big to fit in available memory this could lead to excessive GC pauses and even
+        /// OutOfMemoryError. Use this flag as a hint for Ignite to fetch result set lazily, thus minimizing memory
+        /// consumption at the cost of moderate performance hit.
+        /// </summary>
+        public bool Lazy { get; set; }
 
         /// <summary>
         /// Returns a <see cref="string" /> that represents this instance.

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/config/benchmark-native-sql-cache-select.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-native-sql-cache-select.properties b/modules/yardstick/config/benchmark-native-sql-cache-select.properties
deleted file mode 100644
index cb397db..0000000
--- a/modules/yardstick/config/benchmark-native-sql-cache-select.properties
+++ /dev/null
@@ -1,96 +0,0 @@
-# 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.
-
-#
-# Contains benchmarks for select queries
-#
-
-now0=`date +'%H%M%S'`
-
-# JVM options.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
-
-# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" \
--Xms8g \
--Xmx8g \
--Xloggc:./gc${now0}.log \
--XX:+PrintGCDetails \
--verbose:gc \
--XX:+UseParNewGC \
--XX:+UseConcMarkSweepGC \
--XX:+PrintGCDateStamps \
-"
-
-# Add to JVM_OPTS to generate JFR profile.
-#-XX:+UnlockCommercialFeatures \
-#-XX:+FlightRecorder -XX:StartFlightRecording=delay=300s,duration=120s,filename=#filename#.jfr \
-
-#Ignite version
-ver="RELEASE-"
-
-# List of default probes.
-# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
-BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe
-
-# Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
-
-# Flag which indicates to restart the servers before every benchmark execution.
-RESTART_SERVERS=true
-
-# Probe point writer class name.
-# BENCHMARK_WRITER=
-
-# The benchmark is applicable only for 1 server and 1 driver
-SERVER_HOSTS=127.0.0.1,127.0.0.1
-DRIVER_HOSTS=127.0.0.1
-
-# Remote username.
-# REMOTE_USER=
-
-# Number of nodes, used to wait for the specified number of nodes to start.
-nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
-
-# Backups count.
-b=1
-
-# Warmup.
-w=60
-
-# Duration.
-d=300
-
-# Threads count.
-t=4
-
-# Sync mode.
-sm=PRIMARY_SYNC
-
-# Jobs.
-j=10
-
-# Run configuration which contains all benchmarks.
-# Note that each benchmark is set to run for 300 seconds (5 min) with warm-up set to 60 seconds (1 minute).
-CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds ${ver}sql-select-native-r1-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds ${ver}sql-select-native-r1K-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 2000 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds ${ver}sql-select-native-r2K-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds ${ver}sql-select-native-r1M-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds ${ver}sql-select-native-r1-lazy-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds ${ver}sql-select-native-r1K-lazy-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 2000 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds ${ver}sql-select-native-r2K-lazy-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn ${nodesNum}  -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds ${ver}sql-select-native-r1M-lazy-${b}-backup,\
-"

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/config/benchmark-native-sql-select.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-native-sql-select.properties b/modules/yardstick/config/benchmark-native-sql-select.properties
index 23b78be..0f0b606 100644
--- a/modules/yardstick/config/benchmark-native-sql-select.properties
+++ b/modules/yardstick/config/benchmark-native-sql-select.properties
@@ -51,8 +51,8 @@ RESTART_SERVERS=true
 # BENCHMARK_WRITER=
 
 # The benchmark is applicable only for 1 server and 1 driver
-SERVER_HOSTS=127.0.0.1,127.0.0.1
-DRIVER_HOSTS=127.0.0.1,127.0.0.1,127.0.0.1
+SERVER_HOSTS=127.0.0.1
+DRIVER_HOSTS=127.0.0.1
 
 # Remote username.
 # REMOTE_USER=
@@ -64,7 +64,7 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS}
 b=1
 
 # Warmup.
-w=60
+w=30
 
 # Duration.
 d=300
@@ -81,13 +81,6 @@ j=10
 # Run configuration which contains all benchmarks.
 # Note that each benchmark is set to run for 300 seconds (5 min) with warm-up set to 60 seconds (1 minute).
 CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1 -cl --lazy false,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1K -cl --lazy false,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1M -cl --lazy false,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1-lazy -cl --lazy true,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1K-lazy -cl --lazy true,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1M-lazy -cl --lazy true,\
-"
-CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-backups-${b}-r1 -cl --lazy false -r 10000,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-r1-${b}-backup -cl,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn NativeSqlQueryRangeBenchmark -sn IgniteNode -ds ${ver}sql-select-native-r1000-${b}-backup -cl,\
 "

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/config/ignite-localhost-sql-query-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-localhost-sql-query-config.xml b/modules/yardstick/config/ignite-localhost-sql-query-config.xml
deleted file mode 100644
index 79a4de6..0000000
--- a/modules/yardstick/config/ignite-localhost-sql-query-config.xml
+++ /dev/null
@@ -1,91 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<!--
-    Ignite Spring configuration file to startup grid.
--->
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd">
-    <import resource="ignite-base-config.xml"/>
-
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-ignite.cfg">
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <value>127.0.0.1:47500</value>
-                                <value>127.0.0.1:47501</value>
-                                <value>127.0.0.1:47502</value>
-                                <value>127.0.0.1:47503</value>
-                                <value>127.0.0.1:47504</value>
-                                <value>127.0.0.1:47505</value>
-                                <value>127.0.0.1:47506</value>
-                                <value>127.0.0.1:47507</value>
-                                <value>127.0.0.1:47508</value>
-                                <value>127.0.0.1:47509</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-
-        <property name="communicationSpi">
-            <bean class="org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi">
-                <property name="sharedMemoryPort" value="-1"/>
-            </bean>
-        </property>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="test"/>
-
-                    <property name="backups" value="1"/>
-
-                    <property name="cacheMode" value="PARTITIONED"/>
-
-                    <property name="atomicityMode" value="ATOMIC"/>
-
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="TEST_LONG"/>
-                                <property name="tableName" value="TEST_LONG"/>
-                                <property name="keyFieldName" value="ID"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="ID" value="java.lang.Long"/>
-                                        <entry key="VAL" value="java.lang.Long"/>
-                                    </map>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-            </list>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
index 21442dc..5aec308 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
@@ -17,10 +17,9 @@
 
 package org.apache.ignite.yardstick;
 
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCountDownLatch;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteState;
 import org.apache.ignite.Ignition;
@@ -30,7 +29,6 @@ import org.yardstickframework.BenchmarkConfiguration;
 import org.yardstickframework.BenchmarkDriverAdapter;
 import org.yardstickframework.BenchmarkUtils;
 
-import static org.apache.ignite.events.EventType.EVTS_DISCOVERY;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.yardstickframework.BenchmarkUtils.jcommander;
 import static org.yardstickframework.BenchmarkUtils.println;
@@ -39,8 +37,6 @@ import static org.yardstickframework.BenchmarkUtils.println;
  * Abstract class for Ignite benchmarks.
  */
 public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
-    private static final long WAIT_NODES_TIMEOUT = TimeUnit.SECONDS.toMillis(30);
-
     /** Arguments. */
     protected final IgniteBenchmarkArguments args = new IgniteBenchmarkArguments();
 
@@ -130,30 +126,22 @@ public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
      * @throws Exception If failed.
      */
     private void waitForNodes() throws Exception {
-        IgniteCountDownLatch allNodesReady = ignite().countDownLatch("allNodesReady", 1, false, true);
+        final CountDownLatch nodesStartedLatch = new CountDownLatch(1);
 
-        // wait for condition when all nodes are ready and release distributed barrier.
         ignite().events().localListen(new IgnitePredicate<Event>() {
             @Override public boolean apply(Event gridEvt) {
-                if (nodesStarted()) {
-                    allNodesReady.countDown();
-                    // todo: return false so unregister?
-                }
+                if (nodesStarted())
+                    nodesStartedLatch.countDown();
 
                 return true;
             }
-        }, EVTS_DISCOVERY);
-
-        if (nodesStarted())
-            allNodesReady.countDown();
+        }, EVT_NODE_JOINED);
 
-        // block on distributed barrier till member 0 release it.
-        println(cfg, "Start waiting for cluster to contain " + args.nodes() + ".");
+        if (!nodesStarted()) {
+            println(cfg, "Waiting for " + (args.nodes() - 1) + " nodes to start...");
 
-        //todo: timeouts?
-        allNodesReady.await();
-
-        println(cfg, "Cluster is ready.");
+            nodesStartedLatch.await();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 3bd5e87..3f4fddc 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -289,12 +289,6 @@ public class IgniteBenchmarkArguments {
     @GridToStringInclude
     public long mvccContentionRange = 10_000;
 
-    /** */
-    @Parameter(names = {"--lazy"},
-        arity = 1,
-        description = "Lazy mode for SQL query execution (default true).")
-    private boolean lazy = true;
-
     /**
      * @return {@code True} if need set {@link DataStorageConfiguration}.
      */
@@ -718,13 +712,6 @@ public class IgniteBenchmarkArguments {
         return mvccContentionRange;
     }
 
-    /**
-     * @return Lazy query execution mode.
-     */
-    public boolean isLazy() {
-        return lazy;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(IgniteBenchmarkArguments.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
index 0f99a9d..129e6a4 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
@@ -21,7 +21,6 @@ import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
 import org.yardstickframework.BenchmarkConfiguration;
-import org.yardstickframework.BenchmarkUtils;
 
 import static org.apache.ignite.yardstick.jdbc.JdbcUtils.fillData;
 
@@ -34,7 +33,5 @@ public abstract class AbstractNativeBenchmark extends IgniteAbstractBenchmark {
         super.setUp(cfg);
 
         fillData(cfg, (IgniteEx)ignite(), args.range(), args.atomicMode());
-
-        BenchmarkUtils.println("Lazy mode: " + args.isLazy());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
index 4466f89..81d6c17 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.yardstick.jdbc;
 
-import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.IgniteEx;
@@ -34,46 +33,30 @@ public class JdbcUtils {
      * @param cfg Benchmark configuration.
      * @param ignite Ignite node.
      * @param range Data key range.
-     * @param atomicMode Cache atomic mode.
      */
     public static void fillData(BenchmarkConfiguration cfg,  IgniteEx ignite, long range, CacheAtomicityMode atomicMode) {
-        IgniteSemaphore sem = ignite.semaphore("jdbc-setup", 1, true, true);
+        println(cfg, "Create table...");
 
-        try {
-            if (sem.tryAcquire()) {
-                println(cfg, "Create table...");
+        String withExpr = atomicMode != null ? " WITH \"atomicity=" + atomicMode.name() + "\";" : ";";
 
-                String withExpr = atomicMode != null ? " WITH \"atomicity=" + atomicMode.name() + "\";" : ";";
+        String qry = "CREATE TABLE test_long (id long primary key, val long)" + withExpr;
 
-                String qry = "CREATE TABLE test_long (id long primary key, val long)" + withExpr;
+        println(cfg, "Creating table with schema: " + qry);
 
-                println(cfg, "Creating table with schema: " + qry);
+        ignite.context().query().querySqlFields(
+            new SqlFieldsQuery(qry), true);
 
-                ignite.context().query().querySqlFields(
-                    new SqlFieldsQuery(qry), true);
+        println(cfg, "Populate data...");
 
-                println(cfg, "Populate data...");
+        for (long l = 1; l <= range; ++l) {
+            ignite.context().query().querySqlFields(
+                new SqlFieldsQuery("insert into test_long (id, val) values (?, ?)")
+                    .setArgs(l, l + 1), true);
 
-                for (long l = 1; l <= range; ++l) {
-                    ignite.context().query().querySqlFields(
-                        new SqlFieldsQuery("insert into test_long (id, val) values (?, ?)")
-                            .setArgs(l, l + 1), true);
-
-                    if (l % 10000 == 0)
-                        println(cfg, "Populate " + l);
-                }
-
-                println(cfg, "Finished populating data");
-            }
-            else {
-                // Acquire (wait setup by other client) and immediately release/
-                println(cfg, "Waits for setup...");
-
-                sem.acquire();
-            }
-        }
-        finally {
-            sem.release();
+            if (l % 10000 == 0)
+                println(cfg, "Populate " + l);
         }
+
+        println(cfg, "Finished populating data");
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
deleted file mode 100644
index e361c9a..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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.yardstick.jdbc;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteSemaphore;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
-import org.yardstickframework.BenchmarkConfiguration;
-import org.yardstickframework.BenchmarkUtils;
-
-import static org.apache.ignite.yardstick.jdbc.JdbcUtils.fillData;
-import static org.yardstickframework.BenchmarkUtils.println;
-
-/**
- * Native sql benchmark that performs select operations.
- */
-public class NativeSqlCacheQueryRangeBenchmark extends IgniteAbstractBenchmark {
-    private IgniteCache cache;
-
-    /** {@inheritDoc} */
-    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
-        super.setUp(cfg);
-
-        fillData();
-
-        BenchmarkUtils.println("Lazy mode: " + args.isLazy());
-    }
-
-    /**
-     *
-     */
-    private void fillData() {
-        IgniteEx ignite = (IgniteEx)ignite();
-
-        IgniteSemaphore sem = ignite.semaphore("data-setup", 1, true, true);
-
-        cache = ignite.getOrCreateCache("test").withKeepBinary();
-
-        try {
-            if (sem.tryAcquire()) {
-
-                println(cfg, "Populate data...");
-
-                for (long l = 1; l <= args.range(); ++l) {
-                    cache.query(
-                        new SqlFieldsQuery("insert into test_long(id, val) values (?, ?)")
-                            .setArgs(l, l + 1));
-
-                    if (l % 10000 == 0)
-                        println(cfg, "Populate " + l);
-                }
-
-                println(cfg, "Finished populating data");
-            }
-            else {
-                // Acquire (wait setup by other client) and immediately release/
-                println(cfg, "Waits for setup...");
-
-                sem.acquire();
-            }
-        }
-        finally {
-            sem.release();
-        }
-    }
-
-    /**
-     * Benchmarked action that performs selects and validates results.
-     *
-     * {@inheritDoc}
-     */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        long expRsSize;
-
-        SqlFieldsQuery qry;
-
-        if (args.sqlRange() == 1) {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long WHERE id = ?");
-
-            qry.setArgs(ThreadLocalRandom.current().nextLong(args.range()) + 1);
-
-            expRsSize = 1;
-        }
-        else if (args.sqlRange() <= 0) {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long");
-
-            expRsSize = args.range();
-        }
-        else {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long WHERE id BETWEEN ? AND ?");
-
-            long id = ThreadLocalRandom.current().nextLong(args.range() - args.sqlRange()) + 1;
-            long maxId = id + args.sqlRange() - 1;
-
-            qry.setArgs(id, maxId);
-
-            expRsSize = args.sqlRange();
-        }
-
-        qry.setLazy(args.isLazy());
-
-        long rsSize = 0;
-
-        try (FieldsQueryCursor<List<?>> cursor = cache.query(qry)) {
-            Iterator<List<?>> it = cursor.iterator();
-
-            while (it.hasNext()) {
-                List<?> row = it.next();
-
-                if ((Long)row.get(0) + 1 != (Long)row.get(1))
-                    throw new Exception("Invalid result retrieved");
-
-                rsSize++;
-            }
-        }
-
-        if (rsSize != expRsSize)
-            throw new Exception("Invalid result set size [actual=" + rsSize + ", expected=" + expRsSize + ']');
-
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
index 33630fd..8dcdda7 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.yardstick.jdbc;
 
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
@@ -46,11 +45,6 @@ public class NativeSqlQueryRangeBenchmark extends AbstractNativeBenchmark {
 
             expRsSize = 1;
         }
-        else if (args.sqlRange() <= 0) {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long");
-
-            expRsSize = args.range();
-        }
         else {
             qry = new SqlFieldsQuery("SELECT id, val FROM test_long WHERE id BETWEEN ? AND ?");
 
@@ -62,17 +56,12 @@ public class NativeSqlQueryRangeBenchmark extends AbstractNativeBenchmark {
             expRsSize = args.sqlRange();
         }
 
-        qry.setLazy(args.isLazy());
-
         long rsSize = 0;
 
         try (FieldsQueryCursor<List<?>> cursor = ((IgniteEx)ignite()).context().query()
                 .querySqlFields(qry, false)) {
-            Iterator<List<?>> it = cursor.iterator();
-
-            while (it.hasNext()) {
-                List<?> row = it.next();
 
+            for (List<?> row : cursor) {
                 if ((Long)row.get(0) + 1 != (Long)row.get(1))
                     throw new Exception("Invalid result retrieved");
 


[41/50] [abbrv] ignite git commit: Attempt to make ttl tests more robust.

Posted by sb...@apache.org.
Attempt to make ttl tests more robust.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d4af2138
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d4af2138
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d4af2138

Branch: refs/heads/ignite-gg-14206
Commit: d4af2138f333eaa5a9b183659d32c96d3e165bad
Parents: 1f068e0
Author: sboikov <sb...@apache.org>
Authored: Tue Oct 23 11:03:23 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Oct 23 11:03:50 2018 +0300

----------------------------------------------------------------------
 .../IgniteCacheConfigVariationsFullApiTest.java | 40 ++++++++++++++++----
 1 file changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d4af2138/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
index e449b3b..e2b1a2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
@@ -4212,8 +4212,31 @@ public class IgniteCacheConfigVariationsFullApiTest extends IgniteCacheConfigVar
      * @throws Exception If failed.
      */
     private void checkTtl(boolean inTx, boolean oldEntry) throws Exception {
-        int ttl = 1000;
+        int ttlVals[] = {600, 1000, 3000};
 
+        int i = 0;
+        while (i < ttlVals.length) {
+            try {
+                checkTtl0(inTx, oldEntry, ttlVals[i]);
+                break;
+            }
+            catch (AssertionFailedError e) {
+                if (i < ttlVals.length - 1)
+                    info("Ttl test failed, try execute with increased ttl");
+                else
+                    throw e;
+            }
+            i++;
+        }
+    }
+
+    /**
+     * @param inTx In tx flag.
+     * @param oldEntry {@code True} to check TTL on old entry, {@code false} on new.
+     * @param ttl TTL value.
+     * @throws Exception If failed.
+     */
+    private void checkTtl0(boolean inTx, boolean oldEntry, int ttl) throws Exception {
         final ExpiryPolicy expiry = new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl));
 
         final IgniteCache<String, Integer> c = jcache();
@@ -4409,11 +4432,7 @@ public class IgniteCacheConfigVariationsFullApiTest extends IgniteCacheConfigVar
 
         // Ensure that old TTL and expire time are not longer "visible".
         entryTtl = entryTtl(srvNodeCache, key);
-
-        assertNotNull(entryTtl.get1());
-        assertNotNull(entryTtl.get2());
-        assertEquals(0, (long)entryTtl.get1());
-        assertEquals(0, (long)entryTtl.get2());
+        assertNull(entryTtl);
 
         // Ensure that next update will not pick old expire time.
 
@@ -4430,7 +4449,7 @@ public class IgniteCacheConfigVariationsFullApiTest extends IgniteCacheConfigVar
                 tx.close();
         }
 
-        U.sleep(2000);
+        U.sleep(ttl + 500);
 
         entryTtl = entryTtl(srvNodeCache, key);
 
@@ -6315,6 +6334,13 @@ public class IgniteCacheConfigVariationsFullApiTest extends IgniteCacheConfigVar
 
             entry.unswap();
 
+            if (!entry.hasValue()) {
+                assertEquals(0, entry.ttl());
+                assertEquals(0, entry.expireTime());
+
+                return null;
+            }
+
             IgnitePair<Long> pair = new IgnitePair<>(entry.ttl(), entry.expireTime());
 
             if (!entry.isNear())


[44/50] [abbrv] ignite git commit: IGNITE-8879 Fix blinking baseline node sometimes unable to connect to cluster - Fixes #4893.

Posted by sb...@apache.org.
IGNITE-8879 Fix blinking baseline node sometimes unable to connect to cluster - Fixes #4893.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3f7109ff
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3f7109ff
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3f7109ff

Branch: refs/heads/ignite-gg-14206
Commit: 3f7109ff52bdfc3157987b2230216cc982392eff
Parents: b1584a8
Author: Ivan Daschinskiy <iv...@gmail.com>
Authored: Tue Oct 23 12:33:24 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Tue Oct 23 12:33:24 2018 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 19 ++++-----
 .../GridCacheDatabaseSharedManager.java         | 22 ++++------
 .../IgniteCacheDatabaseSharedManager.java       |  6 ++-
 .../metastorage/IgniteMetaStorageBasicTest.java | 45 ++++++++++++++++++++
 4 files changed, 66 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3f7109ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 0c2cbe2..63d3635 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
@@ -871,16 +871,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      * @throws IgniteCheckedException If failed.
      */
     private IgniteInternalFuture<?> initCachesOnLocalJoin() throws IgniteCheckedException {
-        if (isLocalNodeNotInBaseline()) {
+        boolean baselineNode = isLocalNodeInBaseline();
+
+        if (!baselineNode) {
             cctx.exchange().exchangerBlockingSectionBegin();
 
             try {
                 cctx.cache().cleanupCachesDirectories();
-
-                cctx.database().cleanupCheckpointDirectory();
-
-                if (cctx.wal() != null)
-                    cctx.wal().cleanupWalDirectories();
             }
             finally {
                 cctx.exchange().exchangerBlockingSectionEnd();
@@ -916,7 +913,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             cctx.exchange().exchangerBlockingSectionBegin();
 
             try {
-                cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                cctx.database().readCheckpointAndRestoreMemory(startDescs, !baselineNode);
             }
             finally {
                 cctx.exchange().exchangerBlockingSectionEnd();
@@ -950,12 +947,12 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     * @return {@code true} if local node is not in baseline and {@code false} otherwise.
+     * @return {@code true} if local node is in baseline and {@code false} otherwise.
      */
-    private boolean isLocalNodeNotInBaseline() {
+    private boolean isLocalNodeInBaseline() {
         BaselineTopology topology = cctx.discovery().discoCache().state().baselineTopology();
 
-        return topology!= null && !topology.consistentIds().contains(cctx.localNode().consistentId());
+        return topology != null && topology.consistentIds().contains(cctx.localNode().consistentId());
     }
 
     /**
@@ -1098,7 +1095,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         cctx.exchange().exchangerBlockingSectionBegin();
 
                         try {
-                            cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                            cctx.database().readCheckpointAndRestoreMemory(startDescs, false);
                         }
                         finally {
                             cctx.exchange().exchangerBlockingSectionEnd();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f7109ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 73d65f4..0f37751 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
@@ -645,7 +645,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             checkpointReadLock();
 
             try {
-                restoreMemory(status, true, storePageMem);
+                restoreMemory(status, true, storePageMem, false);
 
                 metaStorage = new MetaStorage(cctx, regCfg, memMetrics, true);
 
@@ -790,7 +790,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /** {@inheritDoc} */
     @Override public void readCheckpointAndRestoreMemory(
-        List<DynamicCacheDescriptor> cachesToStart
+            List<DynamicCacheDescriptor> cachesToStart,
+            boolean restoreMetastorageOnly
     ) throws IgniteCheckedException {
         assert !cctx.localNode().isClient();
 
@@ -819,7 +820,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 (DataRegionMetricsImpl)memMetricsMap.get(METASTORE_DATA_REGION_NAME)
             );
 
-            WALPointer restore = restoreMemory(status);
+            WALPointer restore = restoreMemory(status, restoreMetastorageOnly, (PageMemoryEx) metaStorage.pageMemory(), true);
 
             if (restore == null && !status.endPtr.equals(CheckpointStatus.NULL_PTR)) {
                 throw new StorageException("Restore wal pointer = " + restore + ", while status.endPtr = " +
@@ -1958,24 +1959,17 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * @param status Checkpoint status.
-     * @throws IgniteCheckedException If failed.
-     * @throws StorageException In case I/O error occurred during operations with storage.
-     */
-    @Nullable private WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedException {
-        return restoreMemory(status, false, (PageMemoryEx)metaStorage.pageMemory());
-    }
-
-    /**
-     * @param status Checkpoint status.
      * @param metastoreOnly If {@code True} restores Metastorage only.
      * @param storePageMem Metastore page memory.
+     * @param finalizeCp If {@code True}, finalizes checkpoint on recovery.
      * @throws IgniteCheckedException If failed.
      * @throws StorageException In case I/O error occurred during operations with storage.
      */
     @Nullable private WALPointer restoreMemory(
         CheckpointStatus status,
         boolean metastoreOnly,
-        PageMemoryEx storePageMem
+        PageMemoryEx storePageMem,
+        boolean finalizeCp
     ) throws IgniteCheckedException {
         assert !metastoreOnly || storePageMem != null;
 
@@ -2135,7 +2129,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             }
         }
 
-        if (metastoreOnly)
+        if (!finalizeCp)
             return null;
 
         WALPointer lastReadPtr = restoreBinaryState.lastReadRecordPointer();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f7109ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index 28ce085..db6b987 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -649,9 +649,13 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /**
      * @param cachesToStart Started caches.
+     * @param restoreMetastorageOnly Apply updates only for metastorage.
      * @throws IgniteCheckedException If failed.
      */
-    public void readCheckpointAndRestoreMemory(List<DynamicCacheDescriptor> cachesToStart) throws IgniteCheckedException {
+    public void readCheckpointAndRestoreMemory(
+            List<DynamicCacheDescriptor> cachesToStart,
+            boolean restoreMetastorageOnly
+    ) throws IgniteCheckedException {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f7109ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
index 1837515..e5a53fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
@@ -17,12 +17,14 @@
 package org.apache.ignite.internal.processors.cache.persistence.metastorage;
 
 import java.io.Serializable;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -106,6 +108,34 @@ public class IgniteMetaStorageBasicTest extends GridCommonAbstractTest {
         verifyKeys(ig, KEYS_CNT, KEY_PREFIX, UPDATED_VAL_PREFIX);
     }
 
+    /**
+     * @throws Exception If fails.
+     */
+    public void testRecoveryOfMetastorageWhenNodeNotInBaseline() throws Exception {
+        IgniteEx ig0 = startGrid(0);
+
+        ig0.cluster().active(true);
+
+        final byte KEYS_CNT = 100;
+        final String KEY_PREFIX = "test.key.";
+        final String NEW_VAL_PREFIX = "new.val.";
+        final String UPDATED_VAL_PREFIX = "updated.val.";
+
+        startGrid(1);
+
+        // Disable checkpoints in order to check whether recovery works.
+        forceCheckpoint(grid(1));
+        disableCheckpoints(grid(1));
+
+        loadKeys(grid(1), KEYS_CNT, KEY_PREFIX, NEW_VAL_PREFIX, UPDATED_VAL_PREFIX);
+
+        stopGrid(1, true);
+
+        startGrid(1);
+
+        verifyKeys(grid(1), KEYS_CNT, KEY_PREFIX, UPDATED_VAL_PREFIX);
+    }
+
     /** */
     private void loadKeys(IgniteEx ig,
         byte keysCnt,
@@ -144,4 +174,19 @@ public class IgniteMetaStorageBasicTest extends GridCommonAbstractTest {
             Assert.assertEquals(valPrefix + i, val);
         }
     }
+
+    /**
+     * Disable checkpoints on a specific node.
+     *
+     * @param node Ignite node.h
+     * @throws IgniteCheckedException If failed.
+     */
+    private void disableCheckpoints(Ignite node) throws IgniteCheckedException {
+        assert !node.cluster().localNode().isClient();
+
+        GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)((IgniteEx)node).context()
+                .cache().context().database();
+
+        dbMgr.enableCheckpoints(false).get();
+    }
 }


[47/50] [abbrv] ignite git commit: IGNITE-9632: SQL: support IN statement with constants/params for partition pruning. This closes #4857.

Posted by sb...@apache.org.
IGNITE-9632: SQL: support IN statement with constants/params for partition pruning. This closes #4857.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d669da15
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d669da15
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d669da15

Branch: refs/heads/ignite-gg-14206
Commit: d669da154f79e48a833a754a59606b65159406c3
Parents: d76ff54
Author: devozerov <vo...@gridgain.com>
Authored: Tue Oct 23 15:15:23 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 23 15:15:23 2018 +0300

----------------------------------------------------------------------
 .../query/h2/sql/GridSqlQuerySplitter.java      | 137 +++++++++++--
 .../InOperationExtractPartitionSelfTest.java    | 201 +++++++++++++++++++
 .../query/h2/twostep/JoinSqlTestHelper.java     |   9 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   3 +
 4 files changed, 331 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d669da15/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index ca9c5bb..b19dd14 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Query;
 import org.h2.command.dml.SelectUnion;
+import org.h2.table.Column;
 import org.h2.table.IndexColumn;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
@@ -2342,6 +2343,64 @@ public class GridSqlQuerySplitter {
                 return null;
             }
 
+            case IN: {
+                // Operation should contain at least two children: left (column) and right (const or column).
+                if (op.size() < 2)
+                    return null;
+
+                // Left operand should be column.
+                GridSqlAst left = op.child();
+
+                GridSqlColumn leftCol;
+
+                if (left instanceof GridSqlColumn)
+                    leftCol = (GridSqlColumn)left;
+                else
+                    return null;
+
+                // Can work only with Ignite's tables.
+                if (!(leftCol.column().getTable() instanceof GridH2Table))
+                    return null;
+
+                CacheQueryPartitionInfo[] res = new CacheQueryPartitionInfo[op.size() - 1];
+
+                for (int i = 1; i < op.size(); i++) {
+                    GridSqlAst right = op.child(i);
+
+                    GridSqlConst rightConst;
+                    GridSqlParameter rightParam;
+
+                    if (right instanceof GridSqlConst) {
+                        rightConst = (GridSqlConst)right;
+                        rightParam = null;
+                    }
+                    else if (right instanceof GridSqlParameter) {
+                        rightConst = null;
+                        rightParam = (GridSqlParameter)right;
+                    }
+                    else
+                        // One of members of "IN" list is neither const, nor param, so we do no know it's partition.
+                        // As this is disjunction, not knowing partition of a single element leads to unknown partition
+                        // set globally. Hence, returning null.
+                        return null;
+
+                    CacheQueryPartitionInfo cur = getCacheQueryPartitionInfo(
+                        leftCol.column(),
+                        rightConst,
+                        rightParam,
+                        ctx
+                    );
+
+                    // Same thing as above: single unknown partition in disjunction defeats optimization.
+                    if (cur == null)
+                        return null;
+
+                    res[i - 1] = cur;
+                }
+
+                return res;
+            }
+
             default:
                 return null;
         }
@@ -2362,39 +2421,85 @@ public class GridSqlQuerySplitter {
         GridSqlElement left = op.child(0);
         GridSqlElement right = op.child(1);
 
-        if (!(left instanceof GridSqlColumn))
+        GridSqlColumn leftCol;
+
+        if (left instanceof GridSqlColumn)
+            leftCol = (GridSqlColumn)left;
+        else
             return null;
 
-        if (!(right instanceof GridSqlConst) && !(right instanceof GridSqlParameter))
+        if (!(leftCol.column().getTable() instanceof GridH2Table))
             return null;
 
-        GridSqlColumn column = (GridSqlColumn)left;
+        GridSqlConst rightConst;
+        GridSqlParameter rightParam;
 
-        if (!(column.column().getTable() instanceof GridH2Table))
+        if (right instanceof GridSqlConst) {
+            rightConst = (GridSqlConst)right;
+            rightParam = null;
+        }
+        else if (right instanceof GridSqlParameter) {
+            rightConst = null;
+            rightParam = (GridSqlParameter)right;
+        }
+        else
             return null;
 
-        GridH2Table tbl = (GridH2Table) column.column().getTable();
+        return getCacheQueryPartitionInfo(leftCol.column(), rightConst, rightParam, ctx);
+    }
+
+    /**
+     * Extracts the partition if possible
+     * @param leftCol Column on the lsft side.
+     * @param rightConst Constant on the right side.
+     * @param rightParam Parameter on the right side.
+     * @param ctx Kernal Context.
+     * @return partition info, or {@code null} if none identified
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable private static CacheQueryPartitionInfo getCacheQueryPartitionInfo(
+        Column leftCol,
+        GridSqlConst rightConst,
+        GridSqlParameter rightParam,
+        GridKernalContext ctx
+    ) throws IgniteCheckedException {
+        assert leftCol != null;
+        assert leftCol.getTable() != null;
+        assert leftCol.getTable() instanceof GridH2Table;
+
+        GridH2Table tbl = (GridH2Table)leftCol.getTable();
 
         GridH2RowDescriptor desc = tbl.rowDescriptor();
 
         IndexColumn affKeyCol = tbl.getAffinityKeyColumn();
 
-        int colId = column.column().getColumnId();
+        int colId = leftCol.getColumnId();
 
         if ((affKeyCol == null || colId != affKeyCol.column.getColumnId()) && !desc.isKeyColumn(colId))
             return null;
 
-        if (right instanceof GridSqlConst) {
-            GridSqlConst constant = (GridSqlConst)right;
-
-            return new CacheQueryPartitionInfo(ctx.affinity().partition(tbl.cacheName(),
-                constant.value().getObject()), null, null, -1, -1);
+        if (rightConst != null) {
+            int part = ctx.affinity().partition(tbl.cacheName(), rightConst.value().getObject());
+
+            return new CacheQueryPartitionInfo(
+                part,
+                null,
+                null,
+                -1,
+                -1
+            );
+        }
+        else if (rightParam != null) {
+            return new CacheQueryPartitionInfo(
+                -1,
+                tbl.cacheName(),
+                tbl.getName(),
+                leftCol.getType(),
+                rightParam.index()
+            );
         }
-
-        GridSqlParameter param = (GridSqlParameter) right;
-
-        return new CacheQueryPartitionInfo(-1, tbl.cacheName(), tbl.getName(),
-            column.column().getType(), param.index());
+        else
+            return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d669da15/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/InOperationExtractPartitionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/InOperationExtractPartitionSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/InOperationExtractPartitionSelfTest.java
new file mode 100644
index 0000000..d27fc52
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/InOperationExtractPartitionSelfTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.query.h2.twostep;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.ORG;
+import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.ORG_COUNT;
+
+/** */
+public class InOperationExtractPartitionSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final int NODES_COUNT = 8;
+
+    /** */
+    private static IgniteCache<String, JoinSqlTestHelper.Organization> orgCache;
+
+    /** */
+    private static LongAdder cnt = new LongAdder();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCommunicationSpi(new TcpCommunicationSpi() {
+            /** {@inheritDoc} */
+            @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) {
+                assert msg != null;
+
+                if (GridIoMessage.class.isAssignableFrom(msg.getClass())) {
+                    GridIoMessage gridMsg = (GridIoMessage)msg;
+
+                    if (GridH2QueryRequest.class.isAssignableFrom(gridMsg.message().getClass()))
+                        cnt.increment();
+                }
+
+                super.sendMessage(node, msg, ackC);
+            }
+        });
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(NODES_COUNT, false);
+
+        orgCache = ignite(0).getOrCreateCache(new CacheConfiguration<String, JoinSqlTestHelper.Organization>(ORG)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class)
+        );
+
+        awaitPartitionMapExchange();
+
+        JoinSqlTestHelper.populateDataIntoOrg(orgCache);
+
+        try (FieldsQueryCursor<List<?>> cur = orgCache.query(new SqlFieldsQuery(
+            "SELECT * FROM Organization org WHERE org.id = '" + ORG + 0 + "'"))) {
+
+            assert cur != null;
+
+            List<List<?>> rows = cur.getAll();
+
+            assert rows.size() == 1;
+        }
+
+        try (FieldsQueryCursor<List<?>> cur = orgCache.query(new SqlFieldsQuery(
+            "SELECT * FROM Organization org WHERE org.id = ?").setArgs(ORG + 0))) {
+
+            assert cur != null;
+
+            List<List<?>> rows = cur.getAll();
+
+            assert rows.size() == 1;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        orgCache = null;
+
+        stopAllGrids();
+    }
+
+    /** */
+    public void testAlternativeUsageOfIn(){
+        try (FieldsQueryCursor<List<?>> cur = orgCache.query(new SqlFieldsQuery(
+            "SELECT * FROM Organization org WHERE org._KEY IN (SELECT subOrg._KEY FROM Organization subOrg)"))) {
+
+            assertNotNull(cur);
+
+            List<List<?>> rows = cur.getAll();
+
+            assertEquals(ORG_COUNT, rows.size());
+        }
+    }
+
+    /** */
+    public void testEmptyList() {
+        testInOperator(Collections.emptyList(), null, 0L, NODES_COUNT - 1);
+    }
+
+    /** */
+    public void testSingleValueList() {
+        testInOperator(Collections.singletonList(ORG + 0), null, 1L, 1);
+        testInOperator(Collections.singletonList(ORG + 1), null, 1L, 1);
+        testInOperator(Collections.singletonList(ORG + String.valueOf(ORG_COUNT - 1)), null, 1L, 1);
+        testInOperator(Collections.singletonList("ORG"), null, 0L, 1);
+        testInOperator(Collections.singletonList("?"), new String[] {ORG + 0}, 1L, 1);
+        testInOperator(Collections.singletonList("?"), new String[] {ORG + 2}, 1L, 1);
+        testInOperator(Collections.singletonList("?"), new String[] {ORG + String.valueOf(ORG_COUNT - 1)}, 1L, 1);
+        testInOperator(Collections.singletonList("?"), new String[] {"ORG"}, 0L, 1);
+    }
+
+    /** */
+    public void testMultipleValueList() {
+        testInOperator(Arrays.asList(ORG + 0, ORG + 3, ORG + String.valueOf(ORG_COUNT - 1)), null, 3, 3);
+        testInOperator(Arrays.asList("ORG", ORG + 0, ORG + 4, ORG + String.valueOf(ORG_COUNT - 1)), null, 3, 4);
+        testInOperator(Arrays.asList(ORG + 0, ORG + 5, ORG + String.valueOf(ORG_COUNT - 1), "ORG"), null, 3, 4);
+        testInOperator(Arrays.asList(ORG + 0, ORG + 6, "MID", ORG + String.valueOf(ORG_COUNT - 1), "ORG"), null, 3, 5);
+
+        final List<String> allArgs3 = Arrays.asList("?", "?", "?");
+        final List<String> allArgs4 = Arrays.asList("?", "?", "?", "?");
+
+        testInOperator(allArgs3, new String[] {ORG + 0, ORG + 7, ORG + String.valueOf(ORG_COUNT - 1)}, 3, 3);
+        testInOperator(allArgs4, new String[] {"ORG", ORG + 0, ORG + 8, ORG + String.valueOf(ORG_COUNT - 1)}, 3, 4);
+        testInOperator(allArgs4, new String[] {ORG + 0, ORG + 9, ORG + String.valueOf(ORG_COUNT - 1), "ORG"}, 3, 4);
+        testInOperator(allArgs4, new String[] {ORG + 0, "MID", ORG + String.valueOf(ORG_COUNT - 1), "ORG"}, 2, 4);
+
+        testInOperator(
+            Arrays.asList("?", ORG + 9, ORG + String.valueOf(ORG_COUNT - 1), "?"),
+            new String[] {ORG + 0, "ORG"},
+            3,
+            4
+        );
+        testInOperator(
+            Arrays.asList("?", "?", ORG + String.valueOf(ORG_COUNT - 1), "ORG"),
+            new String[] {ORG + 0, "MID"},
+            2,
+            4
+        );
+    }
+
+    /**
+     *
+     * @param cnst Constants and parameters('?').
+     * @param args Values of parameters.
+     * @param expRes Expected result.
+     * @param maxReq Maximum number of requests to process query.
+     */
+    private void testInOperator(List<String> cnst, Object[] args, long expRes, int maxReq) {
+        int curIdx = cnt.intValue();
+
+        String toIn = cnst.size() == 0 ? "" : String.valueOf("'" + String.join("','", cnst) + "'")
+            .replace("'?'", "?");
+
+        try (FieldsQueryCursor<List<?>> cur = orgCache.query(new SqlFieldsQuery(
+            "SELECT * FROM Organization org WHERE org._KEY IN (" + toIn + ")").setArgs(args))) {
+
+            assertNotNull(cur);
+
+            List<List<?>> rows = cur.getAll();
+
+            assertEquals(expRes, rows.size());
+
+            assertTrue(cnt.intValue() - curIdx <= maxReq);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d669da15/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java
index fe7821a..3c9509d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java
@@ -25,7 +25,10 @@ import org.apache.ignite.cache.query.annotations.QuerySqlField;
  */
 public class JoinSqlTestHelper {
     /** */
-    private static final int ORG_COUNT = 100;
+    static final String ORG = "org";
+
+    /** */
+    static final int ORG_COUNT = 100;
 
     /** */
     private static final int PERSON_PER_ORG_COUNT = 10;
@@ -43,7 +46,7 @@ public class JoinSqlTestHelper {
         for (int i = 0; i < ORG_COUNT; i++) {
             Organization org = new Organization();
 
-            org.setId("org" + i);
+            org.setId(ORG + i);
 
             org.setName("Organization #" + i);
 
@@ -61,7 +64,7 @@ public class JoinSqlTestHelper {
         for (int i = 0; i < ORG_COUNT; i++) {
             Organization org = new Organization();
 
-            org.setId("org" + i);
+            org.setId(ORG + i);
 
             org.setName("Organization #" + i);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d669da15/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index dba046b..ac8d10a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.query.h2.twostep.CacheQueryMemoryLe
 import org.apache.ignite.internal.processors.query.h2.twostep.CreateTableWithDateKeySelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.DisappearedCacheCauseRetryMessageSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.DisappearedCacheWasNotFoundMessageSelfTest;
+import org.apache.ignite.internal.processors.query.h2.twostep.InOperationExtractPartitionSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.NonCollocatedRetryMessageSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.RetryCauseMessageSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.TableViewSubquerySelfTest;
@@ -123,6 +124,8 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
         suite.addTestSuite(DisappearedCacheCauseRetryMessageSelfTest.class);
         suite.addTestSuite(DisappearedCacheWasNotFoundMessageSelfTest.class);
 
+        suite.addTestSuite(InOperationExtractPartitionSelfTest.class);
+
         suite.addTestSuite(TableViewSubquerySelfTest.class);
 
         return suite;


[30/50] [abbrv] ignite git commit: IGNITE-9900: Upgrade annotations.jar to a new version. - Fixes #5004.

Posted by sb...@apache.org.
IGNITE-9900: Upgrade annotations.jar to a new version. - Fixes #5004.

Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ebaedf9a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ebaedf9a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ebaedf9a

Branch: refs/heads/ignite-gg-14206
Commit: ebaedf9a4f1a99c4d9b2c8b3c0affb4494d79c38
Parents: 3fae41b
Author: Stanislav Lukyanov <st...@gmail.com>
Authored: Mon Oct 22 15:08:10 2018 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Oct 22 15:08:10 2018 +0300

----------------------------------------------------------------------
 parent/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebaedf9a/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index eeceb18..e260118 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -80,7 +80,7 @@
         <javax.cache.bundle.version>1.0.0_1</javax.cache.bundle.version>
         <javax.cache.tck.version>1.0.1</javax.cache.tck.version>
         <javax.cache.version>1.0.0</javax.cache.version>
-        <jetbrains.annotations.version>13.0</jetbrains.annotations.version>
+        <jetbrains.annotations.version>16.0.3</jetbrains.annotations.version>
         <jetty.version>9.4.11.v20180605</jetty.version>
         <jmh.version>1.13</jmh.version>
         <jms.spec.version>1.1.1</jms.spec.version>


[08/50] [abbrv] ignite git commit: IGNITE-9868 Improved background full message sending - Fixes #4975.

Posted by sb...@apache.org.
IGNITE-9868 Improved background full message sending - Fixes #4975.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ce73c9d8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ce73c9d8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ce73c9d8

Branch: refs/heads/ignite-gg-14206
Commit: ce73c9d85ffd5b50e4c3370b13302605392fa572
Parents: aec3f91
Author: Sergey Antonov <an...@gmail.com>
Authored: Wed Oct 17 20:08:34 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Oct 17 20:18:36 2018 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      | 177 ++++++++++++++-----
 .../GridDhtPartitionsExchangeFuture.java        |  12 +-
 2 files changed, 135 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ce73c9d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 0baf5a3..6af9678 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
@@ -1028,11 +1028,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * Partition refresh callback.
+     * Partition refresh callback for selected cache groups.
      * For coordinator causes {@link GridDhtPartitionsFullMessage FullMessages} send,
      * for non coordinator -  {@link GridDhtPartitionsSingleMessage SingleMessages} send
+     *
+     * @param grps Cache groups for partitions refresh.
      */
-    public void refreshPartitions() {
+    public void refreshPartitions(@NotNull Collection<CacheGroupContext> grps) {
         // TODO https://issues.apache.org/jira/browse/IGNITE-6857
         if (cctx.snapshot().snapshotOperationInProgress()) {
             scheduleResendPartitions();
@@ -1040,6 +1042,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
         }
 
+        if (grps.isEmpty()) {
+            if (log.isDebugEnabled())
+                log.debug("Skip partitions refresh, there are no cache groups for partition refresh.");
+
+            return;
+        }
+
         ClusterNode oldest = cctx.discovery().oldestAliveServerNode(AffinityTopologyVersion.NONE);
 
         if (oldest == null) {
@@ -1049,8 +1058,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
         }
 
-        if (log.isDebugEnabled())
-            log.debug("Refreshing partitions [oldest=" + oldest.id() + ", loc=" + cctx.localNodeId() + ']');
+        if (log.isDebugEnabled()) {
+            log.debug("Refreshing partitions [oldest=" + oldest.id() + ", loc=" + cctx.localNodeId() +
+                ", cacheGroups= " + grps + ']');
+        }
 
         // If this is the oldest node.
         if (oldest.id().equals(cctx.localNodeId())) {
@@ -1068,50 +1079,66 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             // No need to send to nodes which did not finish their first exchange.
             AffinityTopologyVersion rmtTopVer =
-                lastFut != null ? (lastFut.isDone() ? lastFut.topologyVersion() : lastFut.initialVersion()) : AffinityTopologyVersion.NONE;
+                lastFut != null ?
+                    (lastFut.isDone() ? lastFut.topologyVersion() : lastFut.initialVersion())
+                    : AffinityTopologyVersion.NONE;
 
             Collection<ClusterNode> rmts = cctx.discovery().remoteAliveNodesWithCaches(rmtTopVer);
 
             if (log.isDebugEnabled())
                 log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
 
-            sendAllPartitions(rmts, rmtTopVer);
+            sendAllPartitions(rmts, rmtTopVer, grps);
         }
         else {
             if (log.isDebugEnabled())
                 log.debug("Refreshing local partitions from non-oldest node: " +
                     cctx.localNodeId());
 
-            sendLocalPartitions(oldest, null);
+            sendLocalPartitions(oldest, null, grps);
         }
     }
 
     /**
+     * Partition refresh callback.
+     * For coordinator causes {@link GridDhtPartitionsFullMessage FullMessages} send,
+     * for non coordinator -  {@link GridDhtPartitionsSingleMessage SingleMessages} send
+     */
+    public void refreshPartitions() { refreshPartitions(cctx.cache().cacheGroups()); }
+
+    /**
      * @param nodes Nodes.
      * @param msgTopVer Topology version. Will be added to full message.
+     * @param grps Selected cache groups.
      */
     private void sendAllPartitions(
         Collection<ClusterNode> nodes,
-        AffinityTopologyVersion msgTopVer
+        AffinityTopologyVersion msgTopVer,
+        Collection<CacheGroupContext> grps
     ) {
         long time = System.currentTimeMillis();
 
-        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(true, false, null, null, null, null);
+        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(true, false, null, null, null, null, grps);
 
         m.topologyVersion(msgTopVer);
 
         if (log.isInfoEnabled()) {
             long latency = System.currentTimeMillis() - time;
 
-            if (latency > 100 || log.isDebugEnabled())
-                log.info("Full Message creating for " + msgTopVer + " performed in " + latency + " ms.");
+            if (latency > 50 || log.isDebugEnabled()) {
+                log.info("Finished full message creation [msgTopVer=" + msgTopVer + ", groups=" + grps +
+                    ", latency=" + latency + "ms]");
+            }
         }
 
         if (log.isTraceEnabled())
-            log.trace("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
+            log.trace("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", cacheGroups=" + grps +
+                ", msg=" + m + ']');
 
         time = System.currentTimeMillis();
 
+        Collection<ClusterNode> failedNodes = U.newHashSet(nodes.size());
+
         for (ClusterNode node : nodes) {
             try {
                 assert !node.equals(cctx.localNode());
@@ -1119,22 +1146,34 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
             }
             catch (ClusterTopologyCheckedException ignore) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
-                        node.id() + ", msg=" + m + ']');
+                if (log.isDebugEnabled()) {
+                    log.debug("Failed to send partition update to node because it left grid (will ignore) " +
+                        "[node=" + node.id() + ", msg=" + m + ']');
+                }
             }
             catch (IgniteCheckedException e) {
-                U.warn(log, "Failed to send partitions full message [node=" + node + ", err=" + e + ']');
+                failedNodes.add(node);
+
+                U.warn(log, "Failed to send partitions full message [node=" + node + ", err=" + e + ']', e);
             }
         }
 
-        if (log.isInfoEnabled())
-            log.info("Sending Full Message for " + msgTopVer + " performed in " + (System.currentTimeMillis() - time) + " ms.");
+        if (log.isInfoEnabled()) {
+            long latency = System.currentTimeMillis() - time;
+
+            if (latency > 50 || log.isDebugEnabled()) {
+                log.info("Finished sending full message [msgTopVer=" + msgTopVer + ", groups=" + grps +
+                    (failedNodes.isEmpty() ? "" : (", skipped=" + U.nodeIds(failedNodes))) +
+                    ", latency=" + latency + "ms]");
+            }
+        }
     }
 
     /**
+     * Creates partitions full message for all cache groups.
+     *
      * @param compress {@code True} if possible to compress message (properly work only if prepareMarshall/
-     *     finishUnmarshall methods are called).
+     * finishUnmarshall methods are called).
      * @param newCntrMap {@code True} if possible to use {@link CachePartitionFullCountersMap}.
      * @param exchId Non-null exchange ID if message is created for exchange.
      * @param lastVer Last version.
@@ -1150,18 +1189,43 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         @Nullable IgniteDhtPartitionHistorySuppliersMap partHistSuppliers,
         @Nullable IgniteDhtPartitionsToReloadMap partsToReload
     ) {
-        final GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchId,
-            lastVer,
-            exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE,
-            partHistSuppliers,
-            partsToReload
-            );
+        Collection<CacheGroupContext> grps = cctx.cache().cacheGroups();
+
+        return createPartitionsFullMessage(compress, newCntrMap, exchId, lastVer, partHistSuppliers, partsToReload, grps);
+    }
+
+    /**
+     * Creates partitions full message for selected cache groups.
+     *
+     * @param compress {@code True} if possible to compress message (properly work only if prepareMarshall/
+     *     finishUnmarshall methods are called).
+     * @param newCntrMap {@code True} if possible to use {@link CachePartitionFullCountersMap}.
+     * @param exchId Non-null exchange ID if message is created for exchange.
+     * @param lastVer Last version.
+     * @param partHistSuppliers Partition history suppliers map.
+     * @param partsToReload Partitions to reload map.
+     * @param grps Selected cache groups.
+     * @return Message.
+     */
+    public GridDhtPartitionsFullMessage createPartitionsFullMessage(
+        boolean compress,
+        boolean newCntrMap,
+        @Nullable final GridDhtPartitionExchangeId exchId,
+        @Nullable GridCacheVersion lastVer,
+        @Nullable IgniteDhtPartitionHistorySuppliersMap partHistSuppliers,
+        @Nullable IgniteDhtPartitionsToReloadMap partsToReload,
+        Collection<CacheGroupContext> grps
+    ) {
+        AffinityTopologyVersion ver = exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE;
+
+        final GridDhtPartitionsFullMessage m =
+            new GridDhtPartitionsFullMessage(exchId, lastVer, ver, partHistSuppliers, partsToReload);
 
         m.compress(compress);
 
         final Map<Object, T2<Integer, GridDhtPartitionFullMap>> dupData = new HashMap<>();
 
-        for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
+        for (CacheGroupContext grp : grps) {
             if (!grp.isLocal()) {
                 if (exchId != null) {
                     AffinityTopologyVersion startTopVer = grp.localStartVersion();
@@ -1174,14 +1238,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                 GridDhtPartitionFullMap locMap = grp.topology().partitionMap(true);
 
-                if (locMap != null) {
-                    addFullPartitionsMap(m,
-                        dupData,
-                        compress,
-                        grp.groupId(),
-                        locMap,
-                        affCache.similarAffinityKey());
-                }
+                if (locMap != null)
+                    addFullPartitionsMap(m, dupData, compress, grp.groupId(), locMap, affCache.similarAffinityKey());
 
                 m.addPartitionSizes(grp.groupId(), grp.topology().globalPartSizes());
 
@@ -1202,14 +1260,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
             GridDhtPartitionFullMap map = top.partitionMap(true);
 
-            if (map != null) {
-                addFullPartitionsMap(m,
-                    dupData,
-                    compress,
-                    top.groupId(),
-                    map,
-                    top.similarAffinityKey());
-            }
+            if (map != null)
+                addFullPartitionsMap(m, dupData, compress, top.groupId(), map, top.similarAffinityKey());
 
             if (exchId != null) {
                 CachePartitionFullCountersMap cntrsMap = top.fullUpdateCounters();
@@ -1269,13 +1321,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param node Destination cluster node.
      * @param id Exchange ID.
+     * @param grps Cache groups for send partitions.
      */
-    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
-        GridDhtPartitionsSingleMessage m = createPartitionsSingleMessage(id,
-            cctx.kernalContext().clientNode(),
-            false,
-            false,
-            null);
+    private void sendLocalPartitions(
+        ClusterNode node,
+        @Nullable GridDhtPartitionExchangeId id,
+        @NotNull Collection<CacheGroupContext> grps
+    ) {
+        GridDhtPartitionsSingleMessage m =
+            createPartitionsSingleMessage(id, cctx.kernalContext().clientNode(), false, false, null, grps);
 
         if (log.isTraceEnabled())
             log.trace("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']');
@@ -1294,6 +1348,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
+     * Creates partitions single message for all cache groups.
+     *
      * @param exchangeId Exchange ID.
      * @param clientOnlyExchange Client exchange flag.
      * @param sndCounters {@code True} if need send partition update counters.
@@ -1307,6 +1363,29 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         boolean newCntrMap,
         ExchangeActions exchActions
     ) {
+        Collection<CacheGroupContext> grps = cctx.cache().cacheGroups();
+
+        return createPartitionsSingleMessage(exchangeId, clientOnlyExchange, sndCounters, newCntrMap, exchActions, grps);
+    }
+
+    /**
+     * Creates partitions single message for selected cache groups.
+     *
+     * @param exchangeId Exchange ID.
+     * @param clientOnlyExchange Client exchange flag.
+     * @param sndCounters {@code True} if need send partition update counters.
+     * @param newCntrMap {@code True} if possible to use {@link CachePartitionPartialCountersMap}.
+     * @param grps Selected cache groups.
+     * @return Message.
+     */
+    public GridDhtPartitionsSingleMessage createPartitionsSingleMessage(
+        @Nullable GridDhtPartitionExchangeId exchangeId,
+        boolean clientOnlyExchange,
+        boolean sndCounters,
+        boolean newCntrMap,
+        ExchangeActions exchActions,
+        Collection<CacheGroupContext> grps
+    ) {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(exchangeId,
             clientOnlyExchange,
             cctx.versions().last(),
@@ -1314,7 +1393,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         Map<Object, T2<Integer, GridPartitionStateMap>> dupData = new HashMap<>();
 
-        for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
+        for (CacheGroupContext grp : grps) {
             if (!grp.isLocal() && (exchActions == null || !exchActions.cacheGroupStopping(grp.groupId()))) {
                 GridDhtPartitionMap locMap = grp.topology().localPartitionMap();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ce73c9d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 e550a8b..0fe1a25 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
@@ -2080,22 +2080,24 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             if (centralizedAff || forceAffReassignment) {
                 assert !exchCtx.mergeExchanges();
 
+                Collection<CacheGroupContext> grpToRefresh = U.newHashSet(cctx.cache().cacheGroups().size());
+
                 for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
                     if (grp.isLocal())
                         continue;
 
-                    boolean needRefresh = false;
-
                     try {
-                        needRefresh = grp.topology().initPartitionsWhenAffinityReady(res, this);
+                        if (grp.topology().initPartitionsWhenAffinityReady(res, this))
+                            grpToRefresh.add(grp);
                     }
                     catch (IgniteInterruptedCheckedException e) {
                         U.error(log, "Failed to initialize partitions.", e);
                     }
 
-                    if (needRefresh)
-                        cctx.exchange().refreshPartitions();
                 }
+
+                if (!grpToRefresh.isEmpty())
+                    cctx.exchange().refreshPartitions(grpToRefresh);
             }
 
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {


[11/50] [abbrv] ignite git commit: IGNITE-9927: Reverted changes to CacheContinuousQueryOperationFromCallbackTest until flaky failures are fixed.

Posted by sb...@apache.org.
IGNITE-9927: Reverted changes to CacheContinuousQueryOperationFromCallbackTest until flaky failures are fixed.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a9f37a2e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a9f37a2e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a9f37a2e

Branch: refs/heads/ignite-gg-14206
Commit: a9f37a2e4800df5eefc81345292d9b4a4ab94c67
Parents: 1b69810
Author: devozerov <vo...@gridgain.com>
Authored: Thu Oct 18 12:19:38 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 18 12:19:38 2018 +0300

----------------------------------------------------------------------
 ...ontinuousQueryOperationFromCallbackTest.java | 205 ++++---------------
 1 file changed, 39 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9f37a2e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
index 3cb13bf..0540b43 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
@@ -63,13 +63,10 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -200,60 +197,6 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
     }
 
     /**
-     * @throws Exception If failed.
-     */
-    public void testMvccTxTwoBackupsFilter() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
-
-        doTest(ccfg, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccTxTwoBackupsFilterPrimary() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
-
-        doTest(ccfg, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccTxReplicatedFilter() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED, 0, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
-
-        doTest(ccfg, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccTxTwoBackup() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
-
-        doTest(ccfg, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccTxReplicated() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
-
-        doTest(ccfg, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccTxReplicatedPrimary() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
-
-        doTest(ccfg, true);
-    }
-
-    /**
      * @param ccfg Cache configuration.
      * @throws Exception If failed.
      */
@@ -309,51 +252,34 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
 
                         QueryTestKey key = new QueryTestKey(rnd.nextInt(KEYS));
 
-                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() !=
-                            ATOMIC && rnd.nextBoolean();
+                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() ==
+                            TRANSACTIONAL && rnd.nextBoolean();
 
                         Transaction tx = null;
 
-                        boolean committed = false;
-
-                        while (!committed && !Thread.currentThread().isInterrupted()) {
-                            try {
-                                if (startTx)
-                                    tx = cache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+                        if (startTx)
+                            tx = cache.unwrap(Ignite.class).transactions().txStart();
 
-                                if ((cache.get(key) == null) || rnd.nextBoolean())
-                                    cache.invoke(key, new IncrementTestEntryProcessor());
-                                else {
-                                    QueryTestValue val;
-                                    QueryTestValue newVal;
+                        try {
+                            if ((cache.get(key) == null) || rnd.nextBoolean())
+                                cache.invoke(key, new IncrementTestEntryProcessor());
+                            else {
+                                QueryTestValue val;
+                                QueryTestValue newVal;
 
-                                    do {
-                                        val = cache.get(key);
+                                do {
+                                    val = cache.get(key);
 
-                                        newVal = val == null ?
-                                            new QueryTestValue(0) : new QueryTestValue(val.val1 + 1);
-                                    }
-                                    while (!cache.replace(key, val, newVal));
+                                    newVal = val == null ?
+                                        new QueryTestValue(0) : new QueryTestValue(val.val1 + 1);
                                 }
-
-                                if (tx != null)
-                                    tx.commit();
-
-                                committed = true;
-                            }
-                            catch (Exception e) {
-                                assertTrue(e.getMessage(), e.getMessage() != null &&
-                                    (e.getMessage().contains("Transaction has been rolled back") ||
-                                        e.getMessage().contains("Cannot serialize transaction due to write conflict")));
-
-                                // Wait MVCC updates become visible.
-                                doSleep(50);
-                            }
-                            finally {
-                                if (tx != null)
-                                    tx.close();
+                                while (!cache.replace(key, val, newVal));
                             }
                         }
+                        finally {
+                            if (tx != null)
+                                tx.commit();
+                        }
                     }
                 }
             }, threadCnt, "put-thread");
@@ -376,7 +302,7 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
                     @Override public boolean apply() {
                         return cbCntr.get() >= expCnt;
                     }
-                }, TimeUnit.SECONDS.toMillis(120));
+                }, TimeUnit.SECONDS.toMillis(60));
 
                 assertTrue("Failed to wait events [exp=" + expCnt + ", act=" + cbCntr.get() + "]", res);
 
@@ -393,7 +319,7 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
                     @Override public boolean apply() {
                         return filterCbCntr.get() >= expInvkCnt;
                     }
-                }, TimeUnit.SECONDS.toMillis(120));
+                }, TimeUnit.SECONDS.toMillis(60));
 
                 assertEquals(expInvkCnt, filterCbCntr.get());
 
@@ -481,45 +407,17 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
             if (e.getKey().compareTo(new QueryTestKey(KEYS)) < 0) {
                 IgniteCache<QueryTestKey, QueryTestValue> cache = ignite.cache(cacheName);
 
-                boolean committed = false;
-                Transaction tx = null;
-                boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL_SNAPSHOT;
-
-                while (!committed && !Thread.currentThread().isInterrupted()) {
-                    try {
-                        if (startTx)
-                            tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
-
-                        if (ThreadLocalRandom.current().nextBoolean()) {
-                            Set<QueryTestKey> keys = new LinkedHashSet<>();
+                if (ThreadLocalRandom.current().nextBoolean()) {
+                    Set<QueryTestKey> keys = new LinkedHashSet<>();
 
-                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                                keys.add(new QueryTestKey(key));
+                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                        keys.add(new QueryTestKey(key));
 
-                            cache.invokeAll(keys, new IncrementTestEntryProcessor());
-                        }
-                        else {
-                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                                cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
-                        }
-
-                        if (tx != null)
-                            tx.commit();
-
-                        committed = true;
-                    }
-                    catch (Exception ex) {
-                        assertTrue(ex.getMessage(), ex.getMessage() != null &&
-                            (ex.getMessage().contains("Transaction has been rolled back") ||
-                                ex.getMessage().contains("Cannot serialize transaction due to write conflict")));
-
-                        // Wait MVCC updates become visible.
-                        doSleep(50);
-                    }
-                    finally {
-                        if (tx != null)
-                            tx.close();
-                    }
+                    cache.invokeAll(keys, new IncrementTestEntryProcessor());
+                }
+                else {
+                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                        cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
                 }
 
                 filterCbCntr.incrementAndGet();
@@ -579,42 +477,17 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
                     cntr.incrementAndGet();
 
                     if (cache != null) {
-                        boolean committed = false;
-                        Transaction tx = null;
-                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL_SNAPSHOT;
-
-                        while (!committed && !Thread.currentThread().isInterrupted()) {
-                            try {
-                                if (startTx)
-                                    tx = cache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
-
-                                if (ThreadLocalRandom.current().nextBoolean()) {
-                                    Set<QueryTestKey> keys = new LinkedHashSet<>();
-
-                                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                                        keys.add(new QueryTestKey(key));
-
-                                    cache.invokeAll(keys, new IncrementTestEntryProcessor());
-                                }
-                                else {
-                                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                                        cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
-                                }
+                        if (ThreadLocalRandom.current().nextBoolean()) {
+                            Set<QueryTestKey> keys = new LinkedHashSet<>();
 
-                                if (tx != null)
-                                    tx.commit();
+                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                                keys.add(new QueryTestKey(key));
 
-                                committed = true;
-                            }
-                            catch (Exception ex) {
-                                assertTrue(ex.getMessage(), ex.getMessage() != null &&
-                                    (ex.getMessage().contains("Transaction has been rolled back") ||
-                                        ex.getMessage().contains("Cannot serialize transaction due to write conflict")));
-                            }
-                            finally {
-                                if (tx != null)
-                                    tx.close();
-                            }
+                            cache.invokeAll(keys, new IncrementTestEntryProcessor());
+                        }
+                        else {
+                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                                cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
                         }
                     }
                 }


[32/50] [abbrv] ignite git commit: IGNITE-9945 Entry start version should not omit datacenter ID.

Posted by sb...@apache.org.
IGNITE-9945 Entry start version should not omit datacenter ID.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e5a46727
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e5a46727
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e5a46727

Branch: refs/heads/ignite-gg-14206
Commit: e5a467272f88e5561a4cc30b510a5608a3b5e739
Parents: 4353618
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Oct 22 16:10:47 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 16:10:47 2018 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java            | 2 +-
 .../ignite/internal/processors/cache/dr/GridCacheDrManager.java   | 3 ++-
 .../ignite/internal/processors/cache/dr/GridOsCacheDrManager.java | 2 +-
 3 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e5a46727/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 4b4b6df..d494857 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
@@ -2105,7 +2105,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                 if (drCacheCtx.isDrEnabled()) {
                     try {
-                        drCacheCtx.dr().onExchange(res, exchId.isLeft());
+                        drCacheCtx.dr().onExchange(res, exchId.isLeft(), activateCluster());
                     }
                     catch (IgniteCheckedException e) {
                         U.error(log, "Failed to notify DR: " + e, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e5a46727/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
index f2a4b30..33a52a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
@@ -60,9 +60,10 @@ public interface GridCacheDrManager extends GridCacheManager {
      *
      * @param topVer Topology version.
      * @param left {@code True} if exchange has been caused by node leave.
+     * @param activate {@code True} if exchange has been caused by cluster activation.
      * @throws IgniteCheckedException If failed.
      */
-    public void onExchange(AffinityTopologyVersion topVer, boolean left) throws IgniteCheckedException;
+    public void onExchange(AffinityTopologyVersion topVer, boolean left, boolean activate) throws IgniteCheckedException;
 
     /**
      * @return {@code True} is DR is enabled.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e5a46727/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
index f3c1b23..425e79c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
@@ -78,7 +78,7 @@ public class GridOsCacheDrManager implements GridCacheDrManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void onExchange(AffinityTopologyVersion topVer, boolean left) throws IgniteCheckedException {
+    @Override public void onExchange(AffinityTopologyVersion topVer, boolean left, boolean activate) throws IgniteCheckedException {
         // No-op.
     }
 


[18/50] [abbrv] ignite git commit: IGNITE-9875 Added missing licence

Posted by sb...@apache.org.
IGNITE-9875 Added missing licence


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5427c098
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5427c098
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5427c098

Branch: refs/heads/ignite-gg-14206
Commit: 5427c098be057027afccc031468f1fb2e32ac7e7
Parents: 177d571
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Oct 19 10:39:17 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 19 10:39:17 2018 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsStateValidatorBenchmark.java  | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5427c098/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/GridDhtPartitionsStateValidatorBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/GridDhtPartitionsStateValidatorBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/GridDhtPartitionsStateValidatorBenchmark.java
index 151606d..f3bbcb9 100644
--- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/GridDhtPartitionsStateValidatorBenchmark.java
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/GridDhtPartitionsStateValidatorBenchmark.java
@@ -1,3 +1,20 @@
+/*
+ * 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.benchmarks.jmh.misc;
 
 import com.google.common.collect.Lists;


[45/50] [abbrv] ignite git commit: IGNITE-9951 PHP: Fixes for Date data type

Posted by sb...@apache.org.
IGNITE-9951 PHP: Fixes for Date data type

This closes #5043


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/685c008a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/685c008a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/685c008a

Branch: refs/heads/ignite-gg-14206
Commit: 685c008af3c5c07f02e7d311fdf0c39f6e3941e0
Parents: 3f7109f
Author: Pavel Petroshenko <pa...@petroshenko.com>
Authored: Tue Oct 23 13:35:37 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Tue Oct 23 13:35:37 2018 +0300

----------------------------------------------------------------------
 modules/platforms/php/src/Apache/Ignite/Data/Date.php      | 9 ++++-----
 modules/platforms/php/src/Apache/Ignite/Data/Timestamp.php | 4 +++-
 2 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/685c008a/modules/platforms/php/src/Apache/Ignite/Data/Date.php
----------------------------------------------------------------------
diff --git a/modules/platforms/php/src/Apache/Ignite/Data/Date.php b/modules/platforms/php/src/Apache/Ignite/Data/Date.php
index 356cc36..fa5f394 100644
--- a/modules/platforms/php/src/Apache/Ignite/Data/Date.php
+++ b/modules/platforms/php/src/Apache/Ignite/Data/Date.php
@@ -47,7 +47,8 @@ class Date
      */
     public static function fromDateTime(DateTime $dateTime)
     {
-        return new Date($dateTime->getTimestamp() * 1000);
+        $millis = intval($dateTime->format('u') / 1000);
+        return new Date($dateTime->getTimestamp() * 1000 + $millis);
     }
     
     /**
@@ -57,11 +58,9 @@ class Date
      */
     public function toDateTime(): DateTime
     {
-        $dateTime = new DateTime();
-        $dateTime->setTimestamp($this->getSeconds());
-        return $dateTime;
+        return DateTime::createFromFormat('U.u', number_format($this->getMillis() / 1000, 6, '.', ''));
     }
-    
+
     /**
      * Returns the date value as number of milliseconds elapsed since January 1, 1970, 00:00:00 UTC.
      * 

http://git-wip-us.apache.org/repos/asf/ignite/blob/685c008a/modules/platforms/php/src/Apache/Ignite/Data/Timestamp.php
----------------------------------------------------------------------
diff --git a/modules/platforms/php/src/Apache/Ignite/Data/Timestamp.php b/modules/platforms/php/src/Apache/Ignite/Data/Timestamp.php
index 39ef984..a404d46 100644
--- a/modules/platforms/php/src/Apache/Ignite/Data/Timestamp.php
+++ b/modules/platforms/php/src/Apache/Ignite/Data/Timestamp.php
@@ -49,7 +49,9 @@ class Timestamp extends Date
      */
     public static function fromDateTime(DateTime $dateTime)
     {
-        return new Timestamp($dateTime->getTimestamp() * 1000, 0);
+        $micros = $dateTime->format('u');
+        $millis = intval($micros / 1000);
+        return new Timestamp($dateTime->getTimestamp() * 1000 + $millis, ($micros % 1000) * 1000);
     }
     
     /**


[16/50] [abbrv] ignite git commit: IGNITE-8570 Improved GridToStringLogger - Fixes #4786.

Posted by sb...@apache.org.
IGNITE-8570 Improved GridToStringLogger - Fixes #4786.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a3b624d9
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a3b624d9
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a3b624d9

Branch: refs/heads/ignite-gg-14206
Commit: a3b624d9f34012eab1711dac14dc71af3c5bd9c4
Parents: 5eb871e
Author: pereslegin-pa <xx...@gmail.com>
Authored: Thu Oct 18 19:08:44 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 18 19:08:44 2018 +0300

----------------------------------------------------------------------
 .../ignite/testframework/GridStringLogger.java  |   3 +
 .../testframework/ListeningTestLogger.java      | 205 +++++++++
 .../ignite/testframework/LogListener.java       | 427 ++++++++++++++++++
 .../test/ListeningTestLoggerTest.java           | 428 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 5 files changed, 1066 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a3b624d9/modules/core/src/test/java/org/apache/ignite/testframework/GridStringLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridStringLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridStringLogger.java
index 9056dd6..0b09d3e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridStringLogger.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridStringLogger.java
@@ -26,7 +26,10 @@ import org.jetbrains.annotations.Nullable;
 
 /**
  * Logger which logs to string buffer.
+ *
+ * @deprecated Use {@link ListeningTestLogger} instead.
  */
+@Deprecated
 public class GridStringLogger implements IgniteLogger {
     /** Initial string builder capacity in bytes */
     private static final int INITIAL = 1024 * 33;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3b624d9/modules/core/src/test/java/org/apache/ignite/testframework/ListeningTestLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/ListeningTestLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/ListeningTestLogger.java
new file mode 100644
index 0000000..1b05f4c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/ListeningTestLogger.java
@@ -0,0 +1,205 @@
+/*
+ * 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.testframework;
+
+import java.util.Collection;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.X;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link org.apache.ignite.IgniteLogger} that performs any actions when certain message is logged.
+ * It can be useful in tests to ensure that a specific message was (or was not) printed to the log.
+ */
+public class ListeningTestLogger implements IgniteLogger {
+    /**
+     * If set to {@code true}, enables debug and trace log messages processing.
+     */
+    private final boolean dbg;
+
+    /**
+     * Logger to echo all messages, limited by {@code dbg} flag.
+     */
+    private final IgniteLogger echo;
+
+    /**
+     * Registered log messages listeners.
+     */
+    private final Collection<Consumer<String>> lsnrs = new CopyOnWriteArraySet<>();
+
+    /**
+     * Default constructor.
+     */
+    public ListeningTestLogger() {
+        this(false);
+    }
+
+    /**
+     * @param dbg If set to {@code true}, enables debug and trace log messages processing.
+     */
+    public ListeningTestLogger(boolean dbg) {
+        this(dbg, null);
+    }
+
+    /**
+     * @param dbg If set to {@code true}, enables debug and trace log messages processing.
+     * @param echo Logger to echo all messages, limited by {@code dbg} flag.
+     */
+    public ListeningTestLogger(boolean dbg, @Nullable IgniteLogger echo) {
+        this.dbg = dbg;
+        this.echo = echo;
+    }
+
+    /**
+     * Registers message listener.
+     *
+     * @param lsnr Message listener.
+     */
+    public void registerListener(@NotNull LogListener lsnr) {
+        lsnr.reset();
+
+        lsnrs.add(lsnr);
+    }
+
+    /**
+     * Registers message listener.
+     * <p>
+     * NOTE listener is executed in the thread causing the logging, so it is not recommended to throw any exceptions
+     * from it. Use {@link LogListener} to create message predicates with assertions.
+     *
+     * @param lsnr Message listener.
+     * @see LogListener
+     */
+    public void registerListener(@NotNull Consumer<String> lsnr) {
+        lsnrs.add(lsnr);
+    }
+
+    /**
+     * Unregisters message listener.
+     *
+     * @param lsnr Message listener.
+     */
+    public void unregisterListener(@NotNull Consumer<String> lsnr) {
+        lsnrs.remove(lsnr);
+    }
+
+    /**
+     * Clears all listeners.
+     */
+    public void clearListeners() {
+        lsnrs.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ListeningTestLogger getLogger(Object ctgr) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void trace(String msg) {
+        if (!dbg)
+            return;
+
+        if (echo != null)
+            echo.trace(msg);
+
+        applyListeners(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void debug(String msg) {
+        if (!dbg)
+            return;
+
+        if (echo != null)
+            echo.debug(msg);
+
+        applyListeners(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void info(String msg) {
+        if (echo != null)
+            echo.info(msg);
+
+        applyListeners(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warning(String msg, @Nullable Throwable t) {
+        if (echo != null)
+            echo.warning(msg, t);
+
+        applyListeners(msg);
+
+        if (t != null)
+            applyListeners(X.getFullStackTrace(t));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(String msg, @Nullable Throwable t) {
+        if (echo != null)
+            echo.error(msg, t);
+
+        applyListeners(msg);
+
+        if (t != null)
+            applyListeners(X.getFullStackTrace(t));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isTraceEnabled() {
+        return dbg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDebugEnabled() {
+        return dbg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isInfoEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isQuiet() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String fileName() {
+        return null;
+    }
+
+    /**
+     * Applies listeners whose pattern is found in the message.
+     *
+     * @param msg Message to check.
+     */
+    private void applyListeners(String msg) {
+        if (msg == null)
+            return;
+
+        for (Consumer<String> lsnr : lsnrs)
+            lsnr.accept(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3b624d9/modules/core/src/test/java/org/apache/ignite/testframework/LogListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/LogListener.java b/modules/core/src/test/java/org/apache/ignite/testframework/LogListener.java
new file mode 100644
index 0000000..d349f06
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/LogListener.java
@@ -0,0 +1,427 @@
+/*
+ * 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.testframework;
+
+import java.time.temporal.ValueRange;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The basic listener for custom log contents checking in {@link ListeningTestLogger}.<br><br>
+ *
+ * Supports {@link #matches(String) substring}, {@link #matches(Pattern) regular expression} or
+ * {@link #matches(Predicate) predicate} listeners and the following optional modifiers:
+ * <ul>
+ *  <li>{@link Builder#times times()} sets the exact number of occurrences</li>
+ *  <li>{@link Builder#atLeast atLeast()} sets the minimum number of occurrences</li>
+ *  <li>{@link Builder#atMost atMost()} sets the maximum number of occurrences</li>
+ * </ul>
+ * {@link Builder#atLeast atLeast()} and {@link Builder#atMost atMost()} can be used together.<br><br>
+ *
+ * If the expected number of occurrences is not specified for the listener,
+ * then at least one occurence is expected by default. In other words:<pre>
+ *
+ * {@code LogListener.matches(msg).build();}
+ *
+ * is equivalent to
+ *
+ * {@code LogListener.matches(msg).atLeast(1).build();}
+ * </pre>
+ *
+ * If only the expected maximum number of occurrences is specified, then
+ * the minimum number of entries for successful validation is zero. In other words:<pre>
+ *
+ * {@code LogListener.matches(msg).atMost(10).build();}
+ *
+ * is equivalent to
+ *
+ * {@code LogListener.matches(msg).atLeast(0).atMost(10).build();}
+ * </pre>
+ */
+public abstract class LogListener implements Consumer<String> {
+    /**
+     * Checks that all conditions are met.
+     *
+     * @throws AssertionError If some condition failed.
+     */
+    public abstract void check() throws AssertionError;
+
+    /**
+     * Reset listener state.
+     */
+    abstract void reset();
+
+    /**
+     * Creates new listener builder.
+     *
+     * @param substr Substring to search for in a log message.
+     * @return Log message listener builder.
+     */
+    public static Builder matches(String substr) {
+        return new Builder().andMatches(substr);
+    }
+
+    /**
+     * Creates new listener builder.
+     *
+     * @param regexp Regular expression to search for in a log message.
+     * @return Log message listener builder.
+     */
+    public static Builder matches(Pattern regexp) {
+        return new Builder().andMatches(regexp);
+    }
+
+    /**
+     * Creates new listener builder.
+     *
+     * @param pred Log message predicate.
+     * @return Log message listener builder.
+     */
+    public static Builder matches(Predicate<String> pred) {
+        return new Builder().andMatches(pred);
+    }
+
+    /**
+     * Log listener builder.
+     */
+    public static class Builder {
+        /** */
+        private final CompositeMessageListener lsnr = new CompositeMessageListener();
+
+        /** */
+        private Node prev;
+
+        /**
+         * Add new substring predicate.
+         *
+         * @param substr Substring.
+         * @return current builder instance.
+         */
+        public Builder andMatches(String substr) {
+            addLast(new Node(substr, msg -> {
+                if (substr.isEmpty())
+                    return msg.isEmpty() ? 1 : 0;
+
+                int cnt = 0;
+
+                for (int idx = 0; (idx = msg.indexOf(substr, idx)) != -1; idx++)
+                    ++cnt;
+
+                return cnt;
+            }));
+
+            return this;
+        }
+
+        /**
+         * Add new regular expression predicate.
+         *
+         * @param regexp Regular expression.
+         * @return current builder instance.
+         */
+        public Builder andMatches(Pattern regexp) {
+            addLast(new Node(regexp.toString(), msg -> {
+                int cnt = 0;
+
+                Matcher matcher = regexp.matcher(msg);
+
+                while (matcher.find())
+                    ++cnt;
+
+                return cnt;
+            }));
+
+            return this;
+        }
+
+        /**
+         * Add new log message predicate.
+         *
+         * @param pred Log message predicate.
+         * @return current builder instance.
+         */
+        public Builder andMatches(Predicate<String> pred) {
+            addLast(new Node(null, msg -> pred.test(msg) ? 1 : 0));
+
+            return this;
+        }
+
+        /**
+         * Set expected number of matches.<br>
+         * Each log message may contain several matches that will be counted,
+         * except {@code Predicate} which can have only one match for message.
+         *
+         * @param n Expected number of matches.
+         * @return current builder instance.
+         */
+        public Builder times(int n) {
+            if (prev != null)
+                prev.cnt = n;
+
+            return this;
+        }
+
+        /**
+         * Set expected minimum number of matches.<br>
+         * Each log message may contain several matches that will be counted,
+         * except {@code Predicate} which can have only one match for message.
+         *
+         * @param n Expected number of matches.
+         * @return current builder instance.
+         */
+        public Builder atLeast(int n) {
+            if (prev != null) {
+                prev.min = n;
+
+                prev.cnt = null;
+            }
+
+            return this;
+        }
+
+        /**
+         * Set expected maximum number of matches.<br>
+         * Each log message may contain several matches that will be counted,
+         * except {@code Predicate} which can have only one match for message.
+         *
+         * @param n Expected number of matches.
+         * @return current builder instance.
+         */
+        public Builder atMost(int n) {
+            if (prev != null) {
+                prev.max = n;
+
+                prev.cnt = null;
+            }
+
+            return this;
+        }
+
+        /**
+         * Set custom message for assertion error.
+         *
+         * @param msg Custom message.
+         * @return current builder instance.
+         */
+        public Builder orError(String msg) {
+            if (prev != null)
+                prev.msg = msg;
+
+            return this;
+        }
+
+        /**
+         * Constructs message listener.
+         *
+         * @return Log message listener.
+         */
+        public LogListener build() {
+            addLast(null);
+
+            return lsnr.lsnrs.size() == 1 ? lsnr.lsnrs.get(0) : lsnr;
+        }
+
+        /**
+         * @param node Log listener attributes.
+         */
+        private void addLast(Node node) {
+            if (prev != null)
+                lsnr.add(prev.listener());
+
+            prev = node;
+        }
+
+        /** */
+        private Builder() {}
+
+        /**
+         * Mutable attributes for log listener.
+         */
+        static final class Node {
+            /** */
+            final String subj;
+
+            /** */
+            final Function<String, Integer> func;
+
+            /** */
+            String msg;
+
+            /** */
+            Integer min;
+
+            /** */
+            Integer max;
+
+            /** */
+            Integer cnt;
+
+            /** */
+            Node(String subj, Function<String, Integer> func) {
+                this.subj = subj;
+                this.func = func;
+            }
+
+            /** */
+            LogMessageListener listener() {
+                ValueRange range;
+
+                if (cnt != null)
+                    range = ValueRange.of(cnt, cnt);
+                else if (min == null && max == null)
+                    range = ValueRange.of(1, Integer.MAX_VALUE);
+                else
+                    range = ValueRange.of(min == null ? 0 : min, max == null ? Integer.MAX_VALUE : max);
+
+                return new LogMessageListener(func, range, subj, msg);
+            }
+        }
+    }
+
+    /** */
+    private static class LogMessageListener extends LogListener {
+        /** */
+        private final Function<String, Integer> func;
+
+        /** */
+        private final AtomicReference<Throwable> err = new AtomicReference<>();
+
+        /** */
+        private final AtomicInteger matches = new AtomicInteger();
+
+        /** */
+        private final ValueRange exp;
+
+        /** */
+        private final String subj;
+
+        /** */
+        private final String errMsg;
+
+        /**
+         * @param subj Search subject.
+         * @param exp Expected occurrences.
+         * @param func Function of counting matches in the message.
+         * @param errMsg Custom error message.
+         */
+        private LogMessageListener(
+            @NotNull Function<String, Integer> func,
+            @NotNull ValueRange exp,
+            @Nullable String subj,
+            @Nullable String errMsg
+        ) {
+            this.func = func;
+            this.exp = exp;
+            this.subj = subj == null ? func.toString() : subj;
+            this.errMsg = errMsg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void accept(String msg) {
+            if (err.get() != null)
+                return;
+
+            try {
+                int cnt = func.apply(msg);
+
+                if (cnt > 0)
+                    matches.addAndGet(cnt);
+            } catch (Throwable t) {
+                err.compareAndSet(null, t);
+
+                if (t instanceof VirtualMachineError)
+                    throw t;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check() {
+            errCheck();
+
+            int matchesCnt = matches.get();
+
+            if (!exp.isValidIntValue(matchesCnt)) {
+                String err =  errMsg != null ? errMsg :
+                    "\"" + subj + "\" matches " + matchesCnt + " times, expected: " +
+                        (exp.getMaximum() == exp.getMinimum() ? exp.getMinimum() : exp) + ".";
+
+                throw new AssertionError(err);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override void reset() {
+            matches.set(0);
+        }
+
+        /**
+         * Check that there were no runtime errors.
+         */
+        private void errCheck() {
+            Throwable t = err.get();
+
+            if (t instanceof Error)
+                throw (Error) t;
+
+            if (t instanceof RuntimeException)
+                throw (RuntimeException) t;
+
+            assert t == null : t;
+        }
+    }
+
+    /** */
+    private static class CompositeMessageListener extends LogListener {
+        /** */
+        private final List<LogMessageListener> lsnrs = new ArrayList<>();
+
+        /** {@inheritDoc} */
+        @Override public void check() {
+            for (LogMessageListener lsnr : lsnrs)
+                lsnr.check();
+        }
+
+        /** {@inheritDoc} */
+        @Override void reset() {
+            for (LogMessageListener lsnr : lsnrs)
+                lsnr.reset();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void accept(String msg) {
+            for (LogMessageListener lsnr : lsnrs)
+                lsnr.accept(msg);
+        }
+
+        /**
+         * @param lsnr Listener.
+         */
+        private void add(LogMessageListener lsnr) {
+            lsnrs.add(lsnr);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3b624d9/modules/core/src/test/java/org/apache/ignite/testframework/test/ListeningTestLoggerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/test/ListeningTestLoggerTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/test/ListeningTestLoggerTest.java
new file mode 100644
index 0000000..a888017
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/test/ListeningTestLoggerTest.java
@@ -0,0 +1,428 @@
+/*
+ * 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.testframework.test;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.logger.NullLogger;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+
+/**
+ * Test.
+ */
+@SuppressWarnings("ThrowableNotThrown")
+public class ListeningTestLoggerTest extends GridCommonAbstractTest {
+    /** */
+    private final ListeningTestLogger log = new ListeningTestLogger(false, super.log);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setGridLogger(log);
+
+        return cfg;
+    }
+
+    /**
+     * Basic example of using listening logger - checks that all running instances of Ignite print product version.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIgniteVersionLogging() throws Exception {
+        int gridCnt = 4;
+
+        LogListener lsnr = LogListener.matches(IgniteVersionUtils.VER_STR).atLeast(gridCnt).build();
+
+        log.registerListener(lsnr);
+
+        try {
+            startGridsMultiThreaded(gridCnt);
+
+            lsnr.check();
+        } finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Checks that re-register works fine.
+     */
+    public void testUnregister() {
+        String msg = "catch me";
+
+        LogListener lsnr1 = LogListener.matches(msg).times(1).build();
+        LogListener lsnr2 = LogListener.matches(msg).times(2).build();
+
+        log.registerListener(lsnr1);
+        log.registerListener(lsnr2);
+
+        log.info(msg);
+
+        log.unregisterListener(lsnr1);
+
+        log.info(msg);
+
+        lsnr1.check();
+        lsnr2.check();
+
+        // Repeat these steps to ensure that the state is cleared during registration.
+        log.registerListener(lsnr1);
+        log.registerListener(lsnr2);
+
+        log.info(msg);
+
+        log.unregisterListener(lsnr1);
+
+        log.info(msg);
+
+        lsnr1.check();
+        lsnr2.check();
+    }
+
+    /**
+     * Ensures that listener will be re-registered only once.
+     */
+    public void testRegister() {
+        AtomicInteger cntr = new AtomicInteger();
+
+        LogListener lsnr3 = LogListener.matches(m -> cntr.incrementAndGet() > 0).build();
+
+        log.registerListener(lsnr3);
+        log.registerListener(lsnr3);
+
+        log.info("1");
+
+        assertEquals(1, cntr.get());
+    }
+
+    /**
+     * Checks basic API.
+     */
+    public void testBasicApi() {
+        String errMsg = "Word started with \"a\" not found.";
+
+        LogListener lsnr = LogListener.matches(Pattern.compile("a[a-z]+")).orError(errMsg)
+            .andMatches("Exception message.").andMatches(".java:").build();
+
+        log.registerListener(lsnr);
+
+        log.info("Something new.");
+
+        assertThrows(log(), () -> {
+            lsnr.check();
+
+            return null;
+        }, AssertionError.class, errMsg);
+
+        log.error("There was an error.", new RuntimeException("Exception message."));
+
+        lsnr.check();
+    }
+
+    /**
+     * Checks blank lines matching.
+     */
+    public void testEmptyLine() {
+        LogListener emptyLineLsnr = LogListener.matches("").build();
+
+        log.registerListener(emptyLineLsnr);
+
+        log.info("");
+
+        emptyLineLsnr.check();
+    }
+
+    /** */
+    public void testPredicateExceptions() {
+        LogListener lsnr = LogListener.matches(msg -> {
+            assertFalse(msg.contains("Target"));
+
+            return true;
+        }).build();
+
+        log.registerListener(lsnr);
+
+        log.info("Ignored message.");
+        log.info("Target message.");
+
+        assertThrowsWithCause(lsnr::check, AssertionError.class);
+
+        // Check custom exception.
+        LogListener lsnr2 = LogListener.matches(msg -> {
+            throw new IllegalStateException("Illegal state");
+        }).orError("ignored blah-blah").build();
+
+        log.registerListener(lsnr2);
+
+        log.info("1");
+        log.info("2");
+
+        assertThrowsWithCause(lsnr2::check, IllegalStateException.class);
+    }
+
+    /**
+     * Validates listener range definition.
+     */
+    public void testRange() {
+        String msg = "range";
+
+        LogListener lsnr2 = LogListener.matches(msg).times(2).build();
+        LogListener lsnr2_3 = LogListener.matches(msg).atLeast(2).atMost(3).build();
+
+        log.registerListener(lsnr2);
+        log.registerListener(lsnr2_3);
+
+        log.info(msg);
+        log.info(msg);
+
+        lsnr2.check();
+        lsnr2_3.check();
+
+        log.info(msg);
+
+        assertThrowsWithCause(lsnr2::check, AssertionError.class);
+
+        lsnr2_3.check();
+
+        log.info(msg);
+
+        assertThrowsWithCause(lsnr2_3::check, AssertionError.class);
+    }
+
+    /**
+     * Checks that substring was not found in the log messages.
+     */
+    public void testNotPresent() {
+        String msg = "vacuum";
+
+        LogListener notPresent = LogListener.matches(msg).times(0).build();
+
+        log.registerListener(notPresent);
+
+        log.info("1");
+
+        notPresent.check();
+
+        log.info(msg);
+
+        assertThrowsWithCause(notPresent::check, AssertionError.class);
+    }
+
+    /**
+     * Checks that the substring is found at least twice.
+     */
+    public void testAtLeast() {
+        String msg = "at least";
+
+        LogListener atLeast2 = LogListener.matches(msg).atLeast(2).build();
+
+        log.registerListener(atLeast2);
+
+        log.info(msg);
+
+        assertThrowsWithCause(atLeast2::check, AssertionError.class);
+
+        log.info(msg);
+
+        atLeast2.check();
+    }
+
+    /**
+     * Checks that the substring is found no more than twice.
+     */
+    public void testAtMost() {
+        String msg = "at most";
+
+        LogListener atMost2 = LogListener.matches(msg).atMost(2).build();
+
+        log.registerListener(atMost2);
+
+        atMost2.check();
+
+        log.info(msg);
+        log.info(msg);
+
+        atMost2.check();
+
+        log.info(msg);
+
+        assertThrowsWithCause(atMost2::check, AssertionError.class);
+    }
+
+    /**
+     * Checks that only last value is taken into account.
+     */
+    public void testMultiRange() {
+        String msg = "multi range";
+
+        LogListener atMost3 = LogListener.matches(msg).times(1).times(2).atMost(3).build();
+
+        log.registerListener(atMost3);
+
+        for (int i = 0; i < 6; i++) {
+            if (i < 4)
+                atMost3.check();
+            else
+                assertThrowsWithCause(atMost3::check, AssertionError.class);
+
+            log.info(msg);
+        }
+
+        LogListener lsnr4 = LogListener.matches(msg).atLeast(2).atMost(3).times(4).build();
+
+        log.registerListener(lsnr4);
+
+        for (int i = 1; i < 6; i++) {
+            log.info(msg);
+
+            if (i == 4)
+                lsnr4.check();
+            else
+                assertThrowsWithCause(lsnr4::check, AssertionError.class);
+        }
+    }
+
+    /**
+     * Checks that matches are counted for each message.
+     */
+    public void testMatchesPerMessage() {
+        LogListener lsnr = LogListener.matches("aa").times(4).build();
+
+        log.registerListener(lsnr);
+
+        log.info("aabaab");
+        log.info("abaaab");
+
+        lsnr.check();
+
+        LogListener newLineLsnr = LogListener.matches("\n").times(5).build();
+
+        log.registerListener(newLineLsnr);
+
+        log.info("\n1\n2\n\n3\n");
+
+        newLineLsnr.check();
+
+        LogListener regexpLsnr = LogListener.matches(Pattern.compile("(?i)hi|hello")).times(3).build();
+
+        log.registerListener(regexpLsnr);
+
+        log.info("Hi! Hello!");
+        log.info("Hi folks");
+
+        regexpLsnr.check();
+    }
+
+    /**
+     * Check thread safety.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultithreaded() throws Exception {
+        int iterCnt = 50_000;
+        int threadCnt = 6;
+        int total = threadCnt * iterCnt;
+        int rndNum = ThreadLocalRandom.current().nextInt(iterCnt);
+
+        LogListener lsnr = LogListener.matches("abba").times(total)
+            .andMatches(Pattern.compile("(?i)abba")).times(total * 2)
+            .andMatches("ab").times(total)
+            .andMatches("ba").times(total)
+            .build();
+
+        LogListener mtLsnr = LogListener.matches("abba").build();
+
+        log.registerListener(lsnr);
+
+        GridTestUtils.runMultiThreaded(() -> {
+            for (int i = 0; i < iterCnt; i++) {
+                if (rndNum == i)
+                    log.registerListener(mtLsnr);
+
+                log.info("It is the abba(ABBA) message.");
+            }
+        }, threadCnt, "test-listening-log");
+
+        lsnr.check();
+        mtLsnr.check();
+    }
+
+    /**
+     * Check "echo" logger.
+     */
+    public void testEchoLogger() {
+        IgniteLogger echo = new StringLogger();
+
+        ListeningTestLogger log = new ListeningTestLogger(true, echo);
+
+        log.error("1");
+        log.warning("2");
+        log.info("3");
+        log.debug("4");
+        log.trace("5");
+
+        assertEquals("12345", echo.toString());
+    }
+
+    /** */
+    private static class StringLogger extends NullLogger {
+        /** */
+        private final StringBuilder buf = new StringBuilder();
+
+        /** {@inheritDoc} */
+        @Override public void trace(String msg) {
+            buf.append(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void debug(String msg) {
+            buf.append(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void info(String msg) {
+            buf.append(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void warning(String msg, Throwable t) {
+            buf.append(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void error(String msg, Throwable t) {
+            buf.append(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return buf.toString();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3b624d9/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index ac2bed3..32cd36e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -87,6 +87,7 @@ import org.apache.ignite.startup.properties.NotStringSystemPropertyTest;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.GridAbstractTest;
 import org.apache.ignite.testframework.test.ConfigVariationsTestSuiteBuilderTest;
+import org.apache.ignite.testframework.test.ListeningTestLoggerTest;
 import org.apache.ignite.testframework.test.ParametersTest;
 import org.apache.ignite.testframework.test.VariationsIteratorTest;
 import org.apache.ignite.util.AttributeNodeFilterSelfTest;
@@ -217,6 +218,8 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheRebalanceConfigValidationTest.class);
 
+        suite.addTestSuite(ListeningTestLoggerTest.class);
+
         return suite;
     }
 }


[09/50] [abbrv] ignite git commit: IGNITE-9918: Restore default-config.xml in the build. This closes #5014.

Posted by sb...@apache.org.
IGNITE-9918: Restore default-config.xml in the build. This closes #5014.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b3688ddf
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b3688ddf
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b3688ddf

Branch: refs/heads/ignite-gg-14206
Commit: b3688ddf482e751e9a5626f0f83bcddf843c6aab
Parents: ce73c9d
Author: Peter Ivanov <mr...@gmail.com>
Authored: Thu Oct 18 10:56:34 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 18 10:56:34 2018 +0300

----------------------------------------------------------------------
 config/apache-ignite-lgpl/default-config.xml    | 29 ++++++++++++++++++++
 config/apache-ignite/default-config.xml         | 29 ++++++++++++++++++++
 config/fabric-lgpl/default-config.xml           | 29 --------------------
 config/fabric/default-config.xml                | 29 --------------------
 .../ignite/internal/GridFactorySelfTest.java    |  9 ------
 5 files changed, 58 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b3688ddf/config/apache-ignite-lgpl/default-config.xml
----------------------------------------------------------------------
diff --git a/config/apache-ignite-lgpl/default-config.xml b/config/apache-ignite-lgpl/default-config.xml
new file mode 100644
index 0000000..5da95ba
--- /dev/null
+++ b/config/apache-ignite-lgpl/default-config.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans
+       http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!--
+        Alter configuration below as needed.
+    -->
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"/>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3688ddf/config/apache-ignite/default-config.xml
----------------------------------------------------------------------
diff --git a/config/apache-ignite/default-config.xml b/config/apache-ignite/default-config.xml
new file mode 100644
index 0000000..5da95ba
--- /dev/null
+++ b/config/apache-ignite/default-config.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans
+       http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <!--
+        Alter configuration below as needed.
+    -->
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"/>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3688ddf/config/fabric-lgpl/default-config.xml
----------------------------------------------------------------------
diff --git a/config/fabric-lgpl/default-config.xml b/config/fabric-lgpl/default-config.xml
deleted file mode 100644
index 5da95ba..0000000
--- a/config/fabric-lgpl/default-config.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="
-       http://www.springframework.org/schema/beans
-       http://www.springframework.org/schema/beans/spring-beans.xsd">
-    <!--
-        Alter configuration below as needed.
-    -->
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"/>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3688ddf/config/fabric/default-config.xml
----------------------------------------------------------------------
diff --git a/config/fabric/default-config.xml b/config/fabric/default-config.xml
deleted file mode 100644
index 5da95ba..0000000
--- a/config/fabric/default-config.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="
-       http://www.springframework.org/schema/beans
-       http://www.springframework.org/schema/beans/spring-beans.xsd">
-    <!--
-        Alter configuration below as needed.
-    -->
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"/>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3688ddf/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
index b453858..fd832a0 100644
--- a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
@@ -148,15 +148,6 @@ public class GridFactorySelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testStartFabricDefault() throws Exception {
-        try (Ignite ignite = Ignition.start("config/fabric/default-config.xml")) {
-            log.info("Started: " + ignite.name());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testStartDefault() throws Exception {
         try (Ignite ignite = Ignition.start("config/default-config.xml")) {
             log.info("Started: " + ignite.name());


[04/50] [abbrv] ignite git commit: IGNITE-9710 Exchange worker liveness checking improvements

Posted by sb...@apache.org.
IGNITE-9710 Exchange worker liveness checking improvements

Signed-off-by: Andrey Gura <ag...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2bdc8982
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2bdc8982
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2bdc8982

Branch: refs/heads/ignite-gg-14206
Commit: 2bdc8982796626f41f39f7ae28af12966b04a24f
Parents: e0e02ab
Author: Andrey Kuznetsov <st...@gmail.com>
Authored: Wed Oct 17 17:25:54 2018 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Wed Oct 17 17:25:54 2018 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       |  35 +-
 .../processors/cache/GridCacheMvccManager.java  |   2 +
 .../GridCachePartitionExchangeManager.java      |   7 +
 .../processors/cache/GridCacheProcessor.java    |   6 +
 .../GridDhtPartitionsExchangeFuture.java        | 366 +++++++++++++++----
 5 files changed, 341 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2bdc8982/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
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 9cbceb1..cedbde1 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
@@ -41,7 +41,6 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
@@ -729,6 +728,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                     return;
 
                 aff.clientEventTopologyChange(evts.lastEvent(), evts.topologyVersion());
+
+                cctx.exchange().exchangerUpdateHeartbeat();
             }
         });
     }
@@ -1154,6 +1155,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 }
                 else
                     aff.clientEventTopologyChange(exchFut.firstEvent(), topVer);
+
+                cctx.exchange().exchangerUpdateHeartbeat();
             }
         });
     }
@@ -1174,6 +1177,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                     AffinityTopologyVersion topVer = fut.initialVersion();
 
                     aff.clientEventTopologyChange(fut.firstEvent(), topVer);
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             });
         }
@@ -1318,16 +1323,22 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 @Override public void applyx(CacheGroupDescriptor desc) throws IgniteCheckedException {
                     CacheGroupHolder cache = groupHolder(fut.initialVersion(), desc);
 
-                    if (cache.affinity().lastVersion().equals(AffinityTopologyVersion.NONE))
+                    if (cache.affinity().lastVersion().equals(AffinityTopologyVersion.NONE)) {
                         calculateAndInit(fut.events(), cache.affinity(), fut.initialVersion());
+
+                        cctx.exchange().exchangerUpdateHeartbeat();
+                    }
                 }
             });
         }
         else {
             forAllCacheGroups(false, new IgniteInClosureX<GridAffinityAssignmentCache>() {
                 @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
-                    if (aff.lastVersion().equals(AffinityTopologyVersion.NONE))
+                    if (aff.lastVersion().equals(AffinityTopologyVersion.NONE)) {
                         initAffinity(cachesRegistry.group(aff.groupId()), aff, fut);
+
+                        cctx.exchange().exchangerUpdateHeartbeat();
+                    }
                 }
             });
         }
@@ -1662,6 +1673,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                         CacheGroupHolder grpHolder = groupHolder(topVer, desc);
 
                         calculateAndInit(fut.events(), grpHolder.affinity(), topVer);
+
+                        cctx.exchange().exchangerUpdateHeartbeat();
                     }
                 });
             }
@@ -1791,6 +1804,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                         calculateAndInit(fut.events(), grp.affinity(), topVer);
                 }
             }
+
+            cctx.exchange().exchangerUpdateHeartbeat();
         }
 
         for (int i = 0; i < fetchFuts.size(); i++) {
@@ -1804,6 +1819,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 fut.events().discoveryCache(),
                 cctx.cache().cacheGroup(grpId).affinity(),
                 fetchFut);
+
+            cctx.exchange().exchangerUpdateHeartbeat();
         }
     }
 
@@ -1874,6 +1891,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                     CacheGroupHolder cache = groupHolder(fut.initialVersion(), desc);
 
                     cache.aff.calculate(fut.initialVersion(), fut.events(), fut.events().discoveryCache());
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             });
         }
@@ -1881,6 +1900,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             forAllCacheGroups(false, new IgniteInClosureX<GridAffinityAssignmentCache>() {
                 @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
                     aff.calculate(fut.initialVersion(), fut.events(), fut.events().discoveryCache());
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             });
         }
@@ -1977,6 +1998,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                                 aff.calculate(topVer, fut.events(), fut.events().discoveryCache());
 
                                 affFut.onDone(topVer);
+
+                                cctx.exchange().exchangerUpdateHeartbeat();
                             }
                         });
 
@@ -1999,6 +2022,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 CacheGroupHolder old = grpHolders.put(grpHolder.groupId(), grpHolder);
 
                 assert old == null : old;
+
+                cctx.exchange().exchangerUpdateHeartbeat();
             }
         });
 
@@ -2075,6 +2100,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                         null,
                         grp.rebalanceEnabled(),
                         affCache);
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             });
 
@@ -2111,6 +2138,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                         for (GridDhtPartitionMap map0 : map.values())
                             cache.topology(fut.context().events().discoveryCache()).update(fut.exchangeId(), map0, true);
                     }
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bdc8982/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 690b15a..16324de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -367,6 +367,8 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                     log.debug("Attempted to remove node locks from removed entry in mvcc manager " +
                         "disco callback (will ignore): " + entry);
             }
+
+            cctx.exchange().exchangerUpdateHeartbeat();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bdc8982/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 dbfc3e4..0baf5a3 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
@@ -2258,6 +2258,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
+     * Invokes {@link GridWorker#updateHeartbeat()} for exchange worker.
+     */
+    public void exchangerUpdateHeartbeat() {
+        exchWorker.updateHeartbeat();
+    }
+
+    /**
      * Invokes {@link GridWorker#blockingSectionBegin()} for exchange worker.
      * Should be called from exchange worker thread.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bdc8982/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 68698ec..ec88a93 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
@@ -2026,6 +2026,8 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
                 t.get2(),
                 exchTopVer,
                 false);
+
+            context().exchange().exchangerUpdateHeartbeat();
         }
 
         if (log.isInfoEnabled())
@@ -2064,6 +2066,8 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
                     null,
                     exchTopVer,
                     false);
+
+                context().exchange().exchangerUpdateHeartbeat();
             }
         }
 
@@ -5030,6 +5034,8 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
             // Make sure to remove future before completion.
             pendingFuts.remove(id, this);
 
+            context().exchange().exchangerUpdateHeartbeat();
+
             return super.onDone(res, err);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bdc8982/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 32cd0d4..e550a8b 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
@@ -86,13 +86,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.LocalJoinCachesContext;
 import org.apache.ignite.internal.processors.cache.StateChangeRequest;
 import org.apache.ignite.internal.processors.cache.WalStateAbstractMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.Latch;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 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.cache.distributed.dht.topology.GridDhtPartitionsStateValidator;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.Latch;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -633,10 +633,15 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (newCrd) {
             IgniteInternalFuture<?> fut = cctx.affinity().initCoordinatorCaches(this, false);
 
-            if (fut != null)
+            if (fut != null) {
                 fut.get();
 
+                cctx.exchange().exchangerUpdateHeartbeat();
+            }
+
             cctx.exchange().onCoordinatorInitialized();
+
+            cctx.exchange().exchangerUpdateHeartbeat();
         }
     }
 
@@ -654,7 +659,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         initTs = U.currentTimeMillis();
 
-        U.await(evtLatch);
+        cctx.exchange().exchangerBlockingSectionBegin();
+
+        try {
+            U.await(evtLatch);
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
 
         assert firstDiscoEvt != null : this;
         assert exchId.nodeId().equals(firstDiscoEvt.eventNode().id()) : this;
@@ -680,7 +692,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             exchCtx = new ExchangeContext(crdNode, mvccCrdChange, this);
 
-            cctx.kernalContext().coordinators().onExchangeStart(mvccCrd, exchCtx, crd);
+            cctx.exchange().exchangerBlockingSectionBegin();
+
+            try {
+                cctx.kernalContext().coordinators().onExchangeStart(mvccCrd, exchCtx, crd);
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
 
             assert state == null : state;
 
@@ -812,8 +831,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     assert false;
             }
 
-            if (cctx.localNode().isClient())
-                tryToPerformLocalSnapshotOperation();
+            if (cctx.localNode().isClient()) {
+                cctx.exchange().exchangerBlockingSectionBegin();
+
+                try {
+                    tryToPerformLocalSnapshotOperation();
+                }
+                finally {
+                    cctx.exchange().exchangerBlockingSectionEnd();
+                }
+            }
 
             if (exchLog.isInfoEnabled())
                 exchLog.info("Finished exchange init [topVer=" + topVer + ", crd=" + crdNode + ']');
@@ -845,15 +872,29 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      */
     private IgniteInternalFuture<?> initCachesOnLocalJoin() throws IgniteCheckedException {
         if (isLocalNodeNotInBaseline()) {
-            cctx.cache().cleanupCachesDirectories();
+            cctx.exchange().exchangerBlockingSectionBegin();
 
-            cctx.database().cleanupCheckpointDirectory();
+            try {
+                cctx.cache().cleanupCachesDirectories();
+
+                cctx.database().cleanupCheckpointDirectory();
 
-            if (cctx.wal() != null)
-                cctx.wal().cleanupWalDirectories();
+                if (cctx.wal() != null)
+                    cctx.wal().cleanupWalDirectories();
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
-        cctx.activate();
+        cctx.exchange().exchangerBlockingSectionBegin();
+
+        try {
+            cctx.activate();
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
 
         LocalJoinCachesContext locJoinCtx = exchActions == null ? null : exchActions.localJoinContext();
 
@@ -872,7 +913,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
             }
 
-            cctx.database().readCheckpointAndRestoreMemory(startDescs);
+            cctx.exchange().exchangerBlockingSectionBegin();
+
+            try {
+                cctx.database().readCheckpointAndRestoreMemory(startDescs);
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
         IgniteInternalFuture<?> cachesRegistrationFut = cctx.cache().startCachesOnLocalJoin(initialVersion(), locJoinCtx);
@@ -923,6 +971,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         continue;
 
                     grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false);
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             }
         }
@@ -953,29 +1003,50 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 boolean updateTop = exchId.topologyVersion().equals(grp.localStartVersion());
 
                 if (updateTop && clientTop != null) {
-                    top.update(null,
-                        clientTop.partitionMap(true),
-                        clientTop.fullUpdateCounters(),
-                        Collections.emptySet(),
-                        null,
-                        null);
+                    cctx.exchange().exchangerBlockingSectionBegin();
+
+                    try {
+                        top.update(null,
+                            clientTop.partitionMap(true),
+                            clientTop.fullUpdateCounters(),
+                            Collections.emptySet(),
+                            null,
+                            null);
+                    }
+                    finally {
+                        cctx.exchange().exchangerBlockingSectionEnd();
+                    }
                 }
             }
 
-            top.updateTopologyVersion(
-                this,
-                events().discoveryCache(),
-                mvccCrd,
-                updSeq,
-                cacheGroupStopping(grp.groupId()));
+            cctx.exchange().exchangerBlockingSectionBegin();
+
+            try {
+                top.updateTopologyVersion(
+                    this,
+                    events().discoveryCache(),
+                    mvccCrd,
+                    updSeq,
+                    cacheGroupStopping(grp.groupId()));
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
-        for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
-            top.updateTopologyVersion(this,
-                events().discoveryCache(),
-                mvccCrd,
-                -1,
-                cacheGroupStopping(top.groupId()));
+        cctx.exchange().exchangerBlockingSectionBegin();
+
+        try {
+            for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
+                top.updateTopologyVersion(this,
+                    events().discoveryCache(),
+                    mvccCrd,
+                    -1,
+                    cacheGroupStopping(top.groupId()));
+            }
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
         }
     }
 
@@ -1004,7 +1075,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
 
                 try {
-                    cctx.activate();
+                    cctx.exchange().exchangerBlockingSectionBegin();
+
+                    try {
+                        cctx.activate();
+                    }
+                    finally {
+                        cctx.exchange().exchangerBlockingSectionEnd();
+                    }
 
                     if (!cctx.kernalContext().clientNode()) {
                         List<DynamicCacheDescriptor> startDescs = new ArrayList<>();
@@ -1017,12 +1095,26 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                                 startDescs.add(desc);
                         }
 
-                        cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                        cctx.exchange().exchangerBlockingSectionBegin();
+
+                        try {
+                            cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                        }
+                        finally {
+                            cctx.exchange().exchangerBlockingSectionEnd();
+                        }
                     }
 
                     assert registerCachesFuture == null : "No caches registration should be scheduled before new caches have started.";
 
-                    registerCachesFuture = cctx.affinity().onCacheChangeRequest(this, crd, exchActions);
+                    cctx.exchange().exchangerBlockingSectionBegin();
+
+                    try {
+                        registerCachesFuture = cctx.affinity().onCacheChangeRequest(this, crd, exchActions);
+                    }
+                    finally {
+                        cctx.exchange().exchangerBlockingSectionEnd();
+                    }
 
                     if (log.isInfoEnabled()) {
                         log.info("Successfully activated caches [nodeId=" + cctx.localNodeId() +
@@ -1038,8 +1130,15 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     exchangeLocE = e;
 
                     if (crd) {
-                        synchronized (mux) {
-                            exchangeGlobalExceptions.put(cctx.localNodeId(), e);
+                        cctx.exchange().exchangerBlockingSectionBegin();
+
+                        try {
+                            synchronized (mux) {
+                                exchangeGlobalExceptions.put(cctx.localNodeId(), e);
+                            }
+                        }
+                        finally {
+                            cctx.exchange().exchangerBlockingSectionEnd();
                         }
                     }
                 }
@@ -1051,6 +1150,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         ", topVer=" + initialVersion() + "]");
                 }
 
+                cctx.exchange().exchangerBlockingSectionBegin();
+
                 try {
                     cctx.kernalContext().dataStructures().onDeActivate(cctx.kernalContext());
 
@@ -1076,9 +1177,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                     exchangeLocE = e;
                 }
+                finally {
+                    cctx.exchange().exchangerBlockingSectionEnd();
+                }
             }
         }
         else if (req.activate()) {
+            cctx.exchange().exchangerBlockingSectionBegin();
+
             // TODO: BLT changes on inactive cluster can't be handled easily because persistent storage hasn't been initialized yet.
             try {
                 if (!forceAffReassignment) {
@@ -1101,6 +1207,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                 exchangeLocE = e;
             }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
         return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL;
@@ -1116,6 +1225,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         assert !exchActions.clientOnlyExchange() : exchActions;
 
+        cctx.exchange().exchangerBlockingSectionBegin();
+
         try {
             assert registerCachesFuture == null : "No caches registration should be scheduled before new caches have started.";
 
@@ -1133,6 +1244,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             exchangeGlobalExceptions.put(cctx.localNodeId(), exchangeLocE);
         }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
 
         return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL;
     }
@@ -1213,10 +1327,17 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (crd != null) {
             assert !crd.isLocal() : crd;
 
-            if (!centralizedAff)
-                sendLocalPartitions(crd);
+            cctx.exchange().exchangerBlockingSectionBegin();
 
-            initDone();
+            try {
+                if (!centralizedAff)
+                    sendLocalPartitions(crd);
+
+                initDone();
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
 
             return;
         }
@@ -1226,13 +1347,22 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     GridAffinityAssignmentCache aff = grp.affinity();
 
                     aff.initialize(initialVersion(), aff.idealAssignment());
+
+                    cctx.exchange().exchangerUpdateHeartbeat();
                 }
             }
             else
                 onAllServersLeft();
         }
 
-        onDone(initialVersion());
+        cctx.exchange().exchangerBlockingSectionBegin();
+
+        try {
+            onDone(initialVersion());
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
     }
 
     /**
@@ -1247,13 +1377,27 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             if (grp.isLocal())
                 continue;
 
-            grp.preloader().onTopologyChanged(this);
+            cctx.exchange().exchangerBlockingSectionBegin();
+
+            try {
+                grp.preloader().onTopologyChanged(this);
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
-        cctx.database().releaseHistoryForPreloading();
+        cctx.exchange().exchangerBlockingSectionBegin();
+
+        try {
+            cctx.database().releaseHistoryForPreloading();
 
-        // To correctly rebalance when persistence is enabled, it is necessary to reserve history within exchange.
-        partHistReserved = cctx.database().reserveHistoryForExchange();
+            // To correctly rebalance when persistence is enabled, it is necessary to reserve history within exchange.
+            partHistReserved = cctx.database().reserveHistoryForExchange();
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
 
         // Skipping wait on local join is available when all cluster nodes have the same protocol.
         boolean skipWaitOnLocalJoin = cctx.exchange().latch().canSkipJoiningNodes(initialVersion())
@@ -1288,7 +1432,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             if (topChanged) {
                 // Partition release future is done so we can flush the write-behind store.
-                cacheCtx.store().forceFlush();
+                cctx.exchange().exchangerBlockingSectionBegin();
+
+                try {
+                    cacheCtx.store().forceFlush();
+                }
+                finally {
+                    cctx.exchange().exchangerBlockingSectionEnd();
+                }
             }
         }
 
@@ -1296,7 +1447,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
            In case of persistent store is enabled we first restore partitions presented on disk.
            We need to guarantee that there are no partition state changes logged to WAL before this callback
            to make sure that we correctly restored last actual states. */
-        boolean restored = cctx.database().beforeExchange(this);
+        boolean restored;
+
+        cctx.exchange().exchangerBlockingSectionBegin();
+
+        try {
+            restored = cctx.database().beforeExchange(this);
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
 
         // Pre-create missing partitions using current affinity.
         if (!exchCtx.mergeExchanges()) {
@@ -1305,25 +1465,48 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     continue;
 
                 // It is possible affinity is not initialized yet if node joins to cluster.
-                if (grp.affinity().lastVersion().topologyVersion() > 0)
-                    grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false);
+                if (grp.affinity().lastVersion().topologyVersion() > 0) {
+                    cctx.exchange().exchangerBlockingSectionBegin();
+
+                    try {
+                        grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false);
+                    }
+                    finally {
+                        cctx.exchange().exchangerBlockingSectionEnd();
+                    }
+                }
             }
         }
 
         // After all partitions have been restored and pre-created it's safe to make first checkpoint.
-        if (restored)
-            cctx.database().onStateRestored();
+        if (restored) {
+            cctx.exchange().exchangerBlockingSectionBegin();
+
+            try {
+                cctx.database().onStateRestored();
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
+        }
 
         changeWalModeIfNeeded();
 
-        if (crd.isLocal()) {
-            if (remaining.isEmpty())
-                onAllReceived(null);
-        }
-        else
-            sendPartitions(crd);
+        cctx.exchange().exchangerBlockingSectionBegin();
 
-        initDone();
+        try {
+            if (crd.isLocal()) {
+                if (remaining.isEmpty())
+                    onAllReceived(null);
+            }
+            else
+                sendPartitions(crd);
+
+            initDone();
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
     }
 
     /**
@@ -1356,8 +1539,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     private void changeWalModeIfNeeded() {
         WalStateAbstractMessage msg = firstWalMessage();
 
-        if (msg != null)
-            cctx.walState().onProposeExchange(msg.exchangeMessage());
+        if (msg != null) {
+            cctx.exchange().exchangerBlockingSectionBegin();
+
+            try {
+                cctx.walState().onProposeExchange(msg.exchangeMessage());
+            }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
+        }
     }
 
     /**
@@ -1397,17 +1588,26 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     private void waitPartitionRelease(boolean distributed, boolean doRollback) throws IgniteCheckedException {
         Latch releaseLatch = null;
 
-        // Wait for other nodes only on first phase.
-        if (distributed)
-            releaseLatch = cctx.exchange().latch().getOrCreate(DISTRIBUTED_LATCH_ID, initialVersion());
+        IgniteInternalFuture<?> partReleaseFut;
 
-        IgniteInternalFuture<?> partReleaseFut = cctx.partitionReleaseFuture(initialVersion());
+        cctx.exchange().exchangerBlockingSectionBegin();
 
-        // Assign to class variable so it will be included into toString() method.
-        this.partReleaseFut = partReleaseFut;
+        try {
+            // Wait for other nodes only on first phase.
+            if (distributed)
+                releaseLatch = cctx.exchange().latch().getOrCreate(DISTRIBUTED_LATCH_ID, initialVersion());
+
+            partReleaseFut = cctx.partitionReleaseFuture(initialVersion());
 
-        if (exchId.isLeft())
-            cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
+            // Assign to class variable so it will be included into toString() method.
+            this.partReleaseFut = partReleaseFut;
+
+            if (exchId.isLeft())
+                cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
+        }
+        finally {
+            cctx.exchange().exchangerBlockingSectionEnd();
+        }
 
         if (log.isTraceEnabled())
             log.trace("Before waiting for partition release future: " + this);
@@ -1428,6 +1628,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             // Read txTimeoutOnPME from configuration after every iteration.
             long curTimeout = cfg.getTransactionConfiguration().getTxTimeoutOnPartitionMapExchange();
 
+            cctx.exchange().exchangerBlockingSectionBegin();
+
             try {
                 // This avoids unnessesary waiting for rollback.
                 partReleaseFut.get(curTimeout > 0 && !txRolledBack ?
@@ -1454,6 +1656,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                 throw e;
             }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
         long waitEnd = U.currentTimeMillis();
@@ -1477,6 +1682,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         dumpCnt = 0;
 
         while (true) {
+            cctx.exchange().exchangerBlockingSectionBegin();
+
             try {
                 locksFut.get(waitTimeout, TimeUnit.MILLISECONDS);
 
@@ -1507,6 +1714,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         U.dumpThreads(log);
                 }
             }
+            finally {
+                cctx.exchange().exchangerBlockingSectionEnd();
+            }
         }
 
         if (releaseLatch == null) {
@@ -1560,6 +1770,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 continue;
 
             grp.preloader().unwindUndeploys();
+
+            cctx.exchange().exchangerUpdateHeartbeat();
         }
 
         cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
@@ -1972,12 +2184,15 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (super.onDone(res, err)) {
             afterLsnrCompleteFut.onDone();
 
-            if (log.isDebugEnabled())
+            if (log.isDebugEnabled()) {
                 log.debug("Completed partition exchange [localNode=" + cctx.localNodeId() + ", exchange= " + this +
-                    ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']');
-            else if(log.isInfoEnabled())
-                log.info("Completed partition exchange [localNode=" + cctx.localNodeId() + ", exchange=" + shortInfo() +
-                     ", topVer=" + topologyVersion() + ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']');
+                        ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']');
+            }
+            else if (log.isInfoEnabled()) {
+                log.info("Completed partition exchange [localNode=" + cctx.localNodeId() +
+                        ", exchange=" + shortInfo() + ", topVer=" + topologyVersion() +
+                        ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']');
+            }
 
             initFut.onDone(err == null);
 
@@ -2041,6 +2256,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     (int)(cctx.kernalContext().config().getFailureDetectionTimeout() / 2));
 
                 for (;;) {
+                    cctx.exchange().exchangerBlockingSectionBegin();
+
                     try {
                         registerCachesFut.get(timeout, TimeUnit.SECONDS);
 
@@ -2055,6 +2272,9 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                             "Probably disk is too busy or slow." +
                             "[caches=" + cacheNames + "]");
                     }
+                    finally {
+                        cctx.exchange().exchangerBlockingSectionEnd();
+                    }
                 }
             }
         }
@@ -4010,6 +4230,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             grp.affinity().idealAssignment(affAssignment);
 
             grp.affinity().initialize(initialVersion(), affAssignment);
+
+            cctx.exchange().exchangerUpdateHeartbeat();
         }
     }
 


[14/50] [abbrv] ignite git commit: IGNITE-9082 Throwing checked exception during tx commit without node stopping leads to data corruption - Fixes #4809.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 775b61c..7e04292 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -558,7 +558,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
     /** {@inheritDoc} */
     @SuppressWarnings({"CatchGenericClass"})
-    @Override public final void userCommit() throws IgniteCheckedException {
+    @Override public void userCommit() throws IgniteCheckedException {
         TransactionState state = state();
 
         if (state != COMMITTING) {
@@ -590,7 +590,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
             WALPointer ptr = null;
 
-            Exception err = null;
+            IgniteCheckedException err = null;
 
             cctx.database().checkpointReadLock();
 
@@ -609,176 +609,175 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
                     UUID nodeId = txEntry.nodeId() == null ? this.nodeId : txEntry.nodeId();
 
-                    try {
-                        while (true) {
-                            try {
-                                GridCacheEntryEx cached = txEntry.cached();
+                    while (true) {
+                        try {
+                            GridCacheEntryEx cached = txEntry.cached();
 
-                                // Must try to evict near entries before committing from
-                                // transaction manager to make sure locks are held.
-                                if (!evictNearEntry(txEntry, false)) {
-                                    if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) {
-                                        cached.markObsolete(xidVer);
+                            // Must try to evict near entries before committing from
+                            // transaction manager to make sure locks are held.
+                            if (!evictNearEntry(txEntry, false)) {
+                                if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) {
+                                    cached.markObsolete(xidVer);
 
-                                        break;
-                                    }
+                                    break;
+                                }
 
-                                    if (cached.detached())
-                                        break;
+                                if (cached.detached())
+                                    break;
 
-                                    boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer);
+                                boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer);
 
-                                    boolean metrics = true;
+                                boolean metrics = true;
 
-                                    if (!updateNearCache && cacheCtx.isNear() && txEntry.locallyMapped())
-                                        metrics = false;
+                                if (!updateNearCache && cacheCtx.isNear() && txEntry.locallyMapped())
+                                    metrics = false;
 
-                                    boolean evt = !isNearLocallyMapped(txEntry, false);
+                                boolean evt = !isNearLocallyMapped(txEntry, false);
 
-                                    if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters()))
-                                        txEntry.cached().unswap(false);
+                                if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters()))
+                                    txEntry.cached().unswap(false);
 
-                                    IgniteBiTuple<GridCacheOperation, CacheObject> res = applyTransformClosures(txEntry,
-                                        true, null);
+                                IgniteBiTuple<GridCacheOperation, CacheObject> res = applyTransformClosures(txEntry,
+                                    true, null);
 
-                                    GridCacheVersion dhtVer = null;
+                                GridCacheVersion dhtVer = null;
 
-                                    // For near local transactions we must record DHT version
-                                    // in order to keep near entries on backup nodes until
-                                    // backup remote transaction completes.
-                                    if (cacheCtx.isNear()) {
-                                        if (txEntry.op() == CREATE || txEntry.op() == UPDATE ||
-                                            txEntry.op() == DELETE || txEntry.op() == TRANSFORM)
-                                            dhtVer = txEntry.dhtVersion();
+                                // For near local transactions we must record DHT version
+                                // in order to keep near entries on backup nodes until
+                                // backup remote transaction completes.
+                                if (cacheCtx.isNear()) {
+                                    if (txEntry.op() == CREATE || txEntry.op() == UPDATE ||
+                                        txEntry.op() == DELETE || txEntry.op() == TRANSFORM)
+                                        dhtVer = txEntry.dhtVersion();
 
-                                        if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) &&
-                                            txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) {
-                                            ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
+                                    if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) &&
+                                        txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) {
+                                        ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
 
-                                            if (expiry != null) {
-                                                txEntry.cached().unswap(false);
+                                        if (expiry != null) {
+                                            txEntry.cached().unswap(false);
 
-                                                Duration duration = cached.hasValue() ?
-                                                    expiry.getExpiryForUpdate() : expiry.getExpiryForCreation();
+                                            Duration duration = cached.hasValue() ?
+                                                expiry.getExpiryForUpdate() : expiry.getExpiryForCreation();
 
-                                                txEntry.ttl(CU.toTtl(duration));
-                                            }
+                                            txEntry.ttl(CU.toTtl(duration));
                                         }
                                     }
+                                }
 
-                                    GridCacheOperation op = res.get1();
-                                    CacheObject val = res.get2();
+                                GridCacheOperation op = res.get1();
+                                CacheObject val = res.get2();
 
-                                    // Deal with conflicts.
-                                    GridCacheVersion explicitVer = txEntry.conflictVersion() != null ?
-                                        txEntry.conflictVersion() : writeVersion();
+                                // Deal with conflicts.
+                                GridCacheVersion explicitVer = txEntry.conflictVersion() != null ?
+                                    txEntry.conflictVersion() : writeVersion();
 
-                                    if ((op == CREATE || op == UPDATE) &&
-                                        txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) {
-                                        ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
+                                if ((op == CREATE || op == UPDATE) &&
+                                    txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) {
+                                    ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
 
-                                        if (expiry != null) {
-                                            Duration duration = cached.hasValue() ?
-                                                expiry.getExpiryForUpdate() : expiry.getExpiryForCreation();
+                                    if (expiry != null) {
+                                        Duration duration = cached.hasValue() ?
+                                            expiry.getExpiryForUpdate() : expiry.getExpiryForCreation();
 
-                                            long ttl = CU.toTtl(duration);
+                                        long ttl = CU.toTtl(duration);
 
-                                            txEntry.ttl(ttl);
+                                        txEntry.ttl(ttl);
 
-                                            if (ttl == CU.TTL_ZERO)
-                                                op = DELETE;
-                                        }
+                                        if (ttl == CU.TTL_ZERO)
+                                            op = DELETE;
                                     }
+                                }
 
-                                    boolean conflictNeedResolve = cacheCtx.conflictNeedResolve();
-
-                                    GridCacheVersionConflictContext<?, ?> conflictCtx = null;
-
-                                    if (conflictNeedResolve) {
-                                        IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContext> conflictRes =
-                                            conflictResolve(op, txEntry, val, explicitVer, cached);
+                                boolean conflictNeedResolve = cacheCtx.conflictNeedResolve();
 
-                                        assert conflictRes != null;
+                                GridCacheVersionConflictContext<?, ?> conflictCtx = null;
 
-                                        conflictCtx = conflictRes.get2();
+                                if (conflictNeedResolve) {
+                                    IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContext> conflictRes =
+                                        conflictResolve(op, txEntry, val, explicitVer, cached);
 
-                                        if (conflictCtx.isUseOld())
-                                            op = NOOP;
-                                        else if (conflictCtx.isUseNew()) {
-                                            txEntry.ttl(conflictCtx.ttl());
-                                            txEntry.conflictExpireTime(conflictCtx.expireTime());
-                                        }
-                                        else {
-                                            assert conflictCtx.isMerge();
+                                    assert conflictRes != null;
 
-                                            op = conflictRes.get1();
-                                            val = txEntry.context().toCacheObject(conflictCtx.mergeValue());
-                                            explicitVer = writeVersion();
+                                    conflictCtx = conflictRes.get2();
 
-                                            txEntry.ttl(conflictCtx.ttl());
-                                            txEntry.conflictExpireTime(conflictCtx.expireTime());
-                                        }
+                                    if (conflictCtx.isUseOld())
+                                        op = NOOP;
+                                    else if (conflictCtx.isUseNew()) {
+                                        txEntry.ttl(conflictCtx.ttl());
+                                        txEntry.conflictExpireTime(conflictCtx.expireTime());
                                     }
-                                    else
-                                        // Nullify explicit version so that innerSet/innerRemove will work as usual.
-                                        explicitVer = null;
+                                    else {
+                                        assert conflictCtx.isMerge();
 
-                                    if (sndTransformedVals || conflictNeedResolve) {
-                                        assert sndTransformedVals && cacheCtx.isReplicated() || conflictNeedResolve;
+                                        op = conflictRes.get1();
+                                        val = txEntry.context().toCacheObject(conflictCtx.mergeValue());
+                                        explicitVer = writeVersion();
 
-                                        txEntry.value(val, true, false);
-                                        txEntry.op(op);
-                                        txEntry.entryProcessors(null);
-                                        txEntry.conflictVersion(explicitVer);
+                                        txEntry.ttl(conflictCtx.ttl());
+                                        txEntry.conflictExpireTime(conflictCtx.expireTime());
                                     }
+                                }
+                                else
+                                    // Nullify explicit version so that innerSet/innerRemove will work as usual.
+                                    explicitVer = null;
 
-                                    if (dhtVer == null)
-                                        dhtVer = explicitVer != null ? explicitVer : writeVersion();
+                                if (sndTransformedVals || conflictNeedResolve) {
+                                    assert sndTransformedVals && cacheCtx.isReplicated() || conflictNeedResolve;
 
-                                    if (op == CREATE || op == UPDATE) {
-                                        assert val != null : txEntry;
+                                    txEntry.value(val, true, false);
+                                    txEntry.op(op);
+                                    txEntry.entryProcessors(null);
+                                    txEntry.conflictVersion(explicitVer);
+                                }
 
-                                        GridCacheUpdateTxResult updRes = cached.innerSet(
-                                            this,
-                                            eventNodeId(),
-                                            txEntry.nodeId(),
-                                            val,
-                                            false,
-                                            false,
-                                            txEntry.ttl(),
-                                            evt,
-                                            metrics,
-                                            txEntry.keepBinary(),
-                                            txEntry.hasOldValue(),
-                                            txEntry.oldValue(),
-                                            topVer,
-                                            null,
-                                            cached.detached() ? DR_NONE : drType,
-                                            txEntry.conflictExpireTime(),
-                                            cached.isNear() ? null : explicitVer,
-                                            CU.subjectId(this, cctx),
-                                            resolveTaskName(),
-                                            dhtVer,
-                                            null,
-                                            mvccSnapshot());
-
-                                        if (updRes.success()) {
-                                            txEntry.updateCounter(updRes.updateCounter());
-
-                                            GridLongList waitTxs = updRes.mvccWaitTransactions();
-
-                                            updateWaitTxs(waitTxs);
-                                        }
+                                if (dhtVer == null)
+                                    dhtVer = explicitVer != null ? explicitVer : writeVersion();
+
+                                if (op == CREATE || op == UPDATE) {
+                                    assert val != null : txEntry;
+
+                                    GridCacheUpdateTxResult updRes = cached.innerSet(
+                                        this,
+                                        eventNodeId(),
+                                        txEntry.nodeId(),
+                                        val,
+                                        false,
+                                        false,
+                                        txEntry.ttl(),
+                                        evt,
+                                        metrics,
+                                        txEntry.keepBinary(),
+                                        txEntry.hasOldValue(),
+                                        txEntry.oldValue(),
+                                        topVer,
+                                        null,
+                                        cached.detached() ? DR_NONE : drType,
+                                        txEntry.conflictExpireTime(),
+                                        cached.isNear() ? null : explicitVer,
+                                        CU.subjectId(this, cctx),
+                                        resolveTaskName(),
+                                        dhtVer,
+                                        null,
+                                        mvccSnapshot());
+
+                                    if (updRes.success()) {
+                                        txEntry.updateCounter(updRes.updateCounter());
+
+                                        GridLongList waitTxs = updRes.mvccWaitTransactions();
+
+                                        updateWaitTxs(waitTxs);
+                                    }
 
-                                        if (updRes.loggedPointer() != null)
-                                            ptr = updRes.loggedPointer();
+                                    if (updRes.loggedPointer() != null)
+                                        ptr = updRes.loggedPointer();
 
-                                        if (updRes.success() && updateNearCache) {
-                                            final CacheObject val0 = val;
-                                            final boolean metrics0 = metrics;
-                                            final GridCacheVersion dhtVer0 = dhtVer;
+                                    if (updRes.success() && updateNearCache) {
+                                        final CacheObject val0 = val;
+                                        final boolean metrics0 = metrics;
+                                        final GridCacheVersion dhtVer0 = dhtVer;
 
-                                            updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerSet(
+                                        updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerSet(
                                                 null,
                                                 eventNodeId(),
                                                 nodeId,
@@ -801,46 +800,46 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                                 dhtVer0,
                                                 null,
                                                 mvccSnapshot())
-                                            );
-                                        }
+                                        );
+                                    }
+                                }
+                                else if (op == DELETE) {
+                                    GridCacheUpdateTxResult updRes = cached.innerRemove(
+                                        this,
+                                        eventNodeId(),
+                                        txEntry.nodeId(),
+                                        false,
+                                        evt,
+                                        metrics,
+                                        txEntry.keepBinary(),
+                                        txEntry.hasOldValue(),
+                                        txEntry.oldValue(),
+                                        topVer,
+                                        null,
+                                        cached.detached() ? DR_NONE : drType,
+                                        cached.isNear() ? null : explicitVer,
+                                        CU.subjectId(this, cctx),
+                                        resolveTaskName(),
+                                        dhtVer,
+                                        null,
+                                        mvccSnapshot());
+
+                                    if (updRes.success()) {
+                                        txEntry.updateCounter(updRes.updateCounter());
+
+                                        GridLongList waitTxs = updRes.mvccWaitTransactions();
+
+                                        updateWaitTxs(waitTxs);
                                     }
-                                    else if (op == DELETE) {
-                                        GridCacheUpdateTxResult updRes = cached.innerRemove(
-                                            this,
-                                            eventNodeId(),
-                                            txEntry.nodeId(),
-                                            false,
-                                            evt,
-                                            metrics,
-                                            txEntry.keepBinary(),
-                                            txEntry.hasOldValue(),
-                                            txEntry.oldValue(),
-                                            topVer,
-                                            null,
-                                            cached.detached() ? DR_NONE : drType,
-                                            cached.isNear() ? null : explicitVer,
-                                            CU.subjectId(this, cctx),
-                                            resolveTaskName(),
-                                            dhtVer,
-                                            null,
-                                            mvccSnapshot());
-
-                                        if (updRes.success()) {
-                                            txEntry.updateCounter(updRes.updateCounter());
-
-                                            GridLongList waitTxs = updRes.mvccWaitTransactions();
-
-                                            updateWaitTxs(waitTxs);
-                                        }
 
-                                        if (updRes.loggedPointer() != null)
-                                            ptr = updRes.loggedPointer();
+                                    if (updRes.loggedPointer() != null)
+                                        ptr = updRes.loggedPointer();
 
-                                        if (updRes.success() && updateNearCache) {
-                                            final boolean metrics0 = metrics;
-                                            final GridCacheVersion dhtVer0 = dhtVer;
+                                    if (updRes.success() && updateNearCache) {
+                                        final boolean metrics0 = metrics;
+                                        final GridCacheVersion dhtVer0 = dhtVer;
 
-                                            updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerRemove(
+                                        updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerRemove(
                                                 null,
                                                 eventNodeId(),
                                                 nodeId,
@@ -859,125 +858,78 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                                 dhtVer0,
                                                 null,
                                                 mvccSnapshot())
-                                            );
-                                        }
+                                        );
                                     }
-                                    else if (op == RELOAD) {
-                                        cached.innerReload();
+                                }
+                                else if (op == RELOAD) {
+                                    cached.innerReload();
 
-                                        if (updateNearCache)
-                                            updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerReload());
+                                    if (updateNearCache)
+                                        updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerReload());
+                                }
+                                else if (op == READ) {
+                                    CacheGroupContext grp = cacheCtx.group();
+
+                                    if (grp.persistenceEnabled() && grp.walEnabled() &&
+                                        cctx.snapshot().needTxReadLogging()) {
+                                        ptr = cctx.wal().log(new DataRecord(new DataEntry(
+                                            cacheCtx.cacheId(),
+                                            txEntry.key(),
+                                            val,
+                                            op,
+                                            nearXidVersion(),
+                                            writeVersion(),
+                                            0,
+                                            txEntry.key().partition(),
+                                            txEntry.updateCounter())));
                                     }
-                                    else if (op == READ) {
-                                        CacheGroupContext grp = cacheCtx.group();
-
-                                        if (grp.persistenceEnabled() && grp.walEnabled() &&
-                                            cctx.snapshot().needTxReadLogging()) {
-                                            ptr = cctx.wal().log(new DataRecord(new DataEntry(
-                                                cacheCtx.cacheId(),
-                                                txEntry.key(),
-                                                val,
-                                                op,
-                                                nearXidVersion(),
-                                                writeVersion(),
-                                                0,
-                                                txEntry.key().partition(),
-                                                txEntry.updateCounter())));
-                                        }
 
-                                        ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
+                                    ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
 
-                                        if (expiry != null) {
-                                            Duration duration = expiry.getExpiryForAccess();
+                                    if (expiry != null) {
+                                        Duration duration = expiry.getExpiryForAccess();
 
-                                            if (duration != null)
-                                                cached.updateTtl(null, CU.toTtl(duration));
-                                        }
-
-                                        if (log.isDebugEnabled())
-                                            log.debug("Ignoring READ entry when committing: " + txEntry);
+                                        if (duration != null)
+                                            cached.updateTtl(null, CU.toTtl(duration));
                                     }
-                                    else {
-                                        assert ownsLock(txEntry.cached()) :
-                                            "Transaction does not own lock for group lock entry during  commit [tx=" +
-                                                this + ", txEntry=" + txEntry + ']';
 
-                                        if (conflictCtx == null || !conflictCtx.isUseOld()) {
-                                            if (txEntry.ttl() != CU.TTL_NOT_CHANGED)
-                                                cached.updateTtl(null, txEntry.ttl());
-                                        }
-
-                                        if (log.isDebugEnabled())
-                                            log.debug("Ignoring NOOP entry when committing: " + txEntry);
-                                    }
+                                    if (log.isDebugEnabled())
+                                        log.debug("Ignoring READ entry when committing: " + txEntry);
                                 }
+                                else {
+                                    assert ownsLock(txEntry.cached()) :
+                                        "Transaction does not own lock for group lock entry during  commit [tx=" +
+                                            this + ", txEntry=" + txEntry + ']';
+
+                                    if (conflictCtx == null || !conflictCtx.isUseOld()) {
+                                        if (txEntry.ttl() != CU.TTL_NOT_CHANGED)
+                                            cached.updateTtl(null, txEntry.ttl());
+                                    }
 
-                                // Check commit locks after set, to make sure that
-                                // we are not changing obsolete entries.
-                                // (innerSet and innerRemove will throw an exception
-                                // if an entry is obsolete).
-                                if (txEntry.op() != READ)
-                                    checkCommitLocks(cached);
-
-                                // Break out of while loop.
-                                break;
-                            }
-                            // If entry cached within transaction got removed.
-                            catch (GridCacheEntryRemovedException ignored) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Got removed entry during transaction commit (will retry): " + txEntry);
-
-                                txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topologyVersion()));
+                                    if (log.isDebugEnabled())
+                                        log.debug("Ignoring NOOP entry when committing: " + txEntry);
+                                }
                             }
-                        }
-                    }
-                    catch (Throwable ex) {
-                        // We are about to initiate transaction rollback when tx has started to committing.
-                        // Need to remove version from committed list.
-                        cctx.tm().removeCommittedTx(this);
 
-                        boolean isNodeStopping = X.hasCause(ex, NodeStoppingException.class);
-                        boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class);
+                            // Check commit locks after set, to make sure that
+                            // we are not changing obsolete entries.
+                            // (innerSet and innerRemove will throw an exception
+                            // if an entry is obsolete).
+                            if (txEntry.op() != READ)
+                                checkCommitLocks(cached);
 
-                        IgniteCheckedException err0 = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " +
-                            "(all transaction entries will be invalidated, however there was a window when " +
-                            "entries for this transaction were visible to others): " + this, ex);
-
-                        if (isNodeStopping) {
-                            U.warn(log, "Failed to commit transaction, node is stopping [tx=" + this +
-                                ", err=" + ex + ']');
-                        }
-                        else if (hasInvalidEnvironmentIssue) {
-                            U.warn(log, "Failed to commit transaction, node is in invalid state and will be stopped [tx=" + this +
-                                ", err=" + ex + ']');
+                            // Break out of while loop.
+                            break;
                         }
-                        else
-                            U.error(log, "Commit failed.", err0);
-
-                        COMMIT_ERR_UPD.compareAndSet(this, null, err0);
+                        // If entry cached within transaction got removed.
+                        catch (GridCacheEntryRemovedException ignored) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got removed entry during transaction commit (will retry): " + txEntry);
 
-                        state(UNKNOWN);
-
-                        if (hasInvalidEnvironmentIssue)
-                            cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, ex));
-                        else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or invalidation.
-                            try {
-                                // Courtesy to minimize damage.
-                                uncommit();
-                            }
-                            catch (Throwable ex1) {
-                                U.error(log, "Failed to uncommit transaction: " + this, ex1);
-
-                                if (ex1 instanceof Error)
-                                    throw ex1;
-                            }
+                            txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topologyVersion()));
                         }
-
-                        if (ex instanceof Error)
-                            throw ex;
-
-                        throw err0;
                     }
+
                 }
 
                 // Apply cache sizes only for primary nodes. Update counters were applied on prepare state.
@@ -988,11 +940,32 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                 if (ptr != null && !cctx.tm().logTxRecords())
                     cctx.wal().flush(ptr, false);
             }
-            catch (StorageException e) {
-                err = e;
+            catch (Throwable ex) {
+                // We are about to initiate transaction rollback when tx has started to committing.
+                // Need to remove version from committed list.
+                cctx.tm().removeCommittedTx(this);
+
+                if (X.hasCause(ex, NodeStoppingException.class)) {
+                    U.warn(log, "Failed to commit transaction, node is stopping [tx=" + CU.txString(this) +
+                        ", err=" + ex + ']');
+
+                    return;
+                }
+
+                err = heuristicException(ex);
+
+                COMMIT_ERR_UPD.compareAndSet(this, null, err);
+
+                state(UNKNOWN);
+
+                try {
+                    uncommit();
+                }
+                catch (Throwable e) {
+                    err.addSuppressed(e);
+                }
 
-                throw new IgniteCheckedException("Failed to log transaction record " +
-                    "(transaction will be rolled back): " + this, e);
+                throw err;
             }
             finally {
                 cctx.database().checkpointReadUnlock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
index f31f0e9..ceeb4e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
@@ -24,6 +24,7 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.NoOpFailureHandler;
 import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -79,6 +80,13 @@ public class FailureProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @return @{code True} if a node will be stopped by current handler in near time.
+     */
+    public boolean nodeStopping() {
+        return failureCtx != null && !(hnd instanceof NoOpFailureHandler);
+    }
+
+    /**
      * This method is used to initialize local failure handler if {@link IgniteConfiguration} don't contain configured one.
      *
      * @return Default {@link FailureHandler} implementation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index a7e6e8c..f68ecd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -41,7 +41,6 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.timeout.GridSpiTimeoutObject;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index facea69..e69aff8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -256,7 +256,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
             cfg.setIndexedTypes(idxTypes);
 
         if (cacheMode() == PARTITIONED)
-            cfg.setBackups(1);
+            cfg.setBackups(backups());
 
         return cfg;
     }
@@ -362,6 +362,13 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @return Backups.
+     */
+    protected int backups() {
+        return 1;
+    }
+
+    /**
      * @param idx Index of grid.
      * @return Default cache.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
index 5f2e2ed..3e59c2a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
@@ -26,12 +26,10 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import javax.cache.Cache;
-import junit.framework.TestCase;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.IgniteTransactions;
-import org.apache.ignite.Ignition;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.QueryCursor;
@@ -47,6 +45,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -57,25 +56,19 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Indexing Spi query only test
  */
-public class IndexingSpiQuerySelfTest extends TestCase {
-    public static final String CACHE_NAME = "test-cache";
+public class IndexingSpiQuerySelfTest extends GridCommonAbstractTest {
+    private IndexingSpi indexingSpi;
 
     /** {@inheritDoc} */
-    @Override public void tearDown() throws Exception {
-        Ignition.stopAll(true);
-    }
-
-    /**
-     * @return Configuration.
-     */
-    protected IgniteConfiguration configuration() {
-        IgniteConfiguration cfg = new IgniteConfiguration();
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
         TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(ipFinder);
 
+        cfg.setIndexingSpi(indexingSpi);
         cfg.setDiscoverySpi(disco);
 
         return cfg;
@@ -86,17 +79,22 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         return new CacheConfiguration<>(cacheName);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testSimpleIndexingSpi() throws Exception {
-        IgniteConfiguration cfg = configuration();
-
-        cfg.setIndexingSpi(new MyIndexingSpi());
+        indexingSpi = new MyIndexingSpi();
 
-        Ignite ignite = Ignition.start(cfg);
+        Ignite ignite = startGrid(0);
 
-        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(CACHE_NAME);
+        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
 
         IgniteCache<Integer, Integer> cache = ignite.createCache(ccfg);
 
@@ -113,13 +111,11 @@ public class IndexingSpiQuerySelfTest extends TestCase {
      * @throws Exception If failed.
      */
     public void testIndexingSpiWithDisabledQueryProcessor() throws Exception {
-        IgniteConfiguration cfg = configuration();
-
-        cfg.setIndexingSpi(new MyIndexingSpi());
+        indexingSpi = new MyIndexingSpi();
 
-        Ignite ignite = Ignition.start(cfg);
+        Ignite ignite = startGrid(0);
 
-        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(CACHE_NAME);
+        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
 
         IgniteCache<Integer, Integer> cache = ignite.createCache(ccfg);
 
@@ -136,13 +132,11 @@ public class IndexingSpiQuerySelfTest extends TestCase {
      * @throws Exception If failed.
      */
     public void testBinaryIndexingSpi() throws Exception {
-        IgniteConfiguration cfg = configuration();
+        indexingSpi = new MyBinaryIndexingSpi();
 
-        cfg.setIndexingSpi(new MyBinaryIndexingSpi());
+        Ignite ignite = startGrid(0);
 
-        Ignite ignite = Ignition.start(cfg);
-
-        CacheConfiguration<PersonKey, Person> ccfg = cacheConfiguration(CACHE_NAME);
+        CacheConfiguration<PersonKey, Person> ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
 
         IgniteCache<PersonKey, Person> cache = ignite.createCache(ccfg);
 
@@ -168,13 +162,11 @@ public class IndexingSpiQuerySelfTest extends TestCase {
     public void testNonBinaryIndexingSpi() throws Exception {
         System.setProperty(IgniteSystemProperties.IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI, "true");
 
-        IgniteConfiguration cfg = configuration();
-
-        cfg.setIndexingSpi(new MyIndexingSpi());
+        indexingSpi = new MyIndexingSpi();
 
-        Ignite ignite = Ignition.start(cfg);
+        Ignite ignite = startGrid(0);
 
-        CacheConfiguration<PersonKey, Person> ccfg = cacheConfiguration(CACHE_NAME);
+        CacheConfiguration<PersonKey, Person> ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
 
         IgniteCache<PersonKey, Person> cache = ignite.createCache(ccfg);
 
@@ -198,13 +190,11 @@ public class IndexingSpiQuerySelfTest extends TestCase {
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     public void testIndexingSpiFailure() throws Exception {
-        IgniteConfiguration cfg = configuration();
-
-        cfg.setIndexingSpi(new MyBrokenIndexingSpi());
+        indexingSpi = new MyBrokenIndexingSpi();
 
-        Ignite ignite = Ignition.start(cfg);
+        Ignite ignite = startGrid(0);
 
-        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(CACHE_NAME);
+        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
 
         ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
index e59deed..ca80b13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheAtomicityMode;
@@ -37,61 +42,64 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionState;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-import javax.cache.Cache;
-
 /**
  * Indexing Spi transactional query test
  */
 public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest {
-    /** */
-    private static AtomicInteger cnt;
-
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 4;
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        cnt = new AtomicInteger();
-
-        super.beforeTestsStarted();
-    }
-
-    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
 
-        if (cnt.getAndIncrement() == 0)
-            cfg.setClientMode(true);
-        else {
-            cfg.setIndexingSpi(new MyBrokenIndexingSpi());
+        cfg.setClientMode("client".equals(igniteInstanceName));
+        cfg.setIndexingSpi(new MyBrokenIndexingSpi());
 
-            CacheConfiguration ccfg = cacheConfiguration(igniteInstanceName);
-            ccfg.setName("test-cache");
-            ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration ccfg = cacheConfiguration(igniteInstanceName);
+        ccfg.setName(DEFAULT_CACHE_NAME);
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
 
-            ccfg.setIndexedTypes(Integer.class, Integer.class);
+        ccfg.setIndexedTypes(Integer.class, Integer.class);
+
+        cfg.setCacheConfiguration(ccfg);
 
-            cfg.setCacheConfiguration(ccfg);
-        }
         return cfg;
     }
 
+    /** */
+    public void testIndexingSpiWithTxClient() throws Exception {
+        IgniteEx client = startGrid("client");
+
+        assertNotNull(client.cache(DEFAULT_CACHE_NAME));
+
+        doTestIndexingSpiWithTx(client, 0);
+    }
+
+    /** */
+    public void testIndexingSpiWithTxLocal() throws Exception {
+        IgniteEx ignite = (IgniteEx)primaryNode(0, DEFAULT_CACHE_NAME);
+
+        doTestIndexingSpiWithTx(ignite, 0);
+    }
+
+    /** */
+    public void testIndexingSpiWithTxNotLocal() throws Exception {
+        IgniteEx ignite = (IgniteEx)primaryNode(0, DEFAULT_CACHE_NAME);
+
+        doTestIndexingSpiWithTx(ignite, 1);
+    }
+
     /**
      * @throws Exception If failed.
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testIndexingSpiWithTx() throws Exception {
-        IgniteEx ignite = grid(0);
-
-        final IgniteCache<Integer, Integer> cache = ignite.cache("test-cache");
+    private void doTestIndexingSpiWithTx(IgniteEx ignite, int key) throws Exception {
+        final IgniteCache<Integer, Integer> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         final IgniteTransactions txs = ignite.transactions();
 
@@ -104,7 +112,7 @@ public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest {
                         Transaction tx;
 
                         try (Transaction tx0 = tx = txs.txStart(concurrency, isolation)) {
-                            cache.put(1, 1);
+                            cache.put(key, key);
 
                             tx0.commit();
                         }
@@ -114,6 +122,8 @@ public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest {
                         return null;
                     }
                 }, IgniteTxHeuristicCheckedException.class);
+
+                checkFutures();
             }
         }
     }
@@ -135,7 +145,7 @@ public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest {
         /** {@inheritDoc} */
         @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
-           return null;
+            return null;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractTransactionIntergrityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractTransactionIntergrityTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractTransactionIntergrityTest.java
index fe27e6e..01db747 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractTransactionIntergrityTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractTransactionIntergrityTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cluster.ClusterNode;
@@ -40,10 +41,10 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.failure.FailureHandler;
 import org.apache.ignite.failure.StopNodeFailureHandler;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.TestRecordingCommunicationSpi;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -73,7 +74,7 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
     private static final int DFLT_ACCOUNTS_CNT = 32;
 
     /** Count of threads and caches. */
-    private static final int DFLT_TX_THREADS_CNT = 20;
+    private static final int DFLT_TX_THREADS_CNT = Runtime.getRuntime().availableProcessors();
 
     /** Count of nodes to start. */
     private static final int DFLT_NODES_CNT = 3;
@@ -126,16 +127,6 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
         return true;
     }
 
-    /**
-     * @return Flag enables cross-node transactions,
-     *         when primary partitions participating in transaction spreaded across several cluster nodes.
-     */
-    protected boolean crossNodeTransactions() {
-        // Commit error during cross node transactions breaks transaction integrity
-        // TODO: https://issues.apache.org/jira/browse/IGNITE-9086
-        return false;
-    }
-
     /** {@inheritDoc} */
     @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
         return new StopNodeFailureHandler();
@@ -148,14 +139,15 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
         cfg.setConsistentId(name);
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
-        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
-        cfg.setLocalHost("127.0.0.1");
 
         cfg.setDataStorageConfiguration(new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
-                .setMaxSize(256 * 1024 * 1024)
-                .setPersistenceEnabled(persistent()))
-        );
+                    .setPersistenceEnabled(persistent())
+                    .setMaxSize(50 * 1024 * 1024)
+            )
+            .setWalSegmentSize(16 * 1024 * 1024)
+            .setPageSize(1024)
+            .setWalMode(WALMode.LOG_ONLY));
 
         CacheConfiguration[] cacheConfigurations = new CacheConfiguration[txThreadsCount()];
 
@@ -178,6 +170,8 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
         cfg.setCacheConfiguration(cacheConfigurations);
 
+        cfg.setFailureDetectionTimeout(30_000);
+
         return cfg;
     }
 
@@ -219,8 +213,11 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
     /**
      * Test transfer amount.
+     *
+     * @param failoverScenario Scenario.
+     * @param colocatedAccounts {@code True} to use colocated on same primary node accounts.
      */
-    public void doTestTransferAmount(FailoverScenario failoverScenario) throws Exception {
+    public void doTestTransferAmount(FailoverScenario failoverScenario, boolean colocatedAccounts) throws Exception {
         failoverScenario.beforeNodesStarted();
 
         //given: started some nodes with client.
@@ -230,26 +227,26 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
         igniteClient.cluster().active(true);
 
-        int[] initAmount = new int[txThreadsCount()];
+        int[] initAmounts = new int[txThreadsCount()];
         completedTxs = new ConcurrentLinkedHashMap[txThreadsCount()];
 
         //and: fill all accounts on all caches and calculate total amount for every cache.
         for (int cachePrefixIdx = 0; cachePrefixIdx < txThreadsCount(); cachePrefixIdx++) {
             IgniteCache<Integer, AccountState> cache = igniteClient.getOrCreateCache(cacheName(cachePrefixIdx));
 
-            AtomicInteger coinsCounter = new AtomicInteger();
+            AtomicInteger coinsCntr = new AtomicInteger();
 
             try (Transaction tx = igniteClient.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                 for (int accountId = 0; accountId < accountsCount(); accountId++) {
-                    Set<Integer> initialAmount = generateCoins(coinsCounter, 5);
+                    Set<Integer> initAmount = generateCoins(coinsCntr, 5);
 
-                    cache.put(accountId, new AccountState(accountId, tx.xid(), initialAmount));
+                    cache.put(accountId, new AccountState(accountId, tx.xid(), initAmount));
                 }
 
                 tx.commit();
             }
 
-            initAmount[cachePrefixIdx] = coinsCounter.get();
+            initAmounts[cachePrefixIdx] = coinsCntr.get();
             completedTxs[cachePrefixIdx] = new ConcurrentLinkedHashMap();
         }
 
@@ -259,7 +256,8 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
         ArrayList<Thread> transferThreads = new ArrayList<>();
 
         for (int i = 0; i < txThreadsCount(); i++) {
-            transferThreads.add(new TransferAmountTxThread(firstTransactionDone, igniteClient, cacheName(i), i));
+            transferThreads.add(new TransferAmountTxThread(firstTransactionDone,
+                igniteClient, cacheName(i), i, colocatedAccounts));
 
             transferThreads.get(i).start();
         }
@@ -268,13 +266,12 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
         failoverScenario.afterFirstTransaction();
 
-        for (Thread thread : transferThreads) {
+        for (Thread thread : transferThreads)
             thread.join();
-        }
 
         failoverScenario.afterTransactionsFinished();
 
-        consistencyCheck(initAmount);
+        consistencyCheck(initAmounts);
     }
 
     /**
@@ -385,11 +382,11 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
         /**
          * @param txId Transaction id.
-         * @param coinsToRemove Coins to remove from current account.
+         * @param coinsToRmv Coins to remove from current account.
          * @return Account state with removed coins.
          */
-        public AccountState removeCoins(IgniteUuid txId, Set<Integer> coinsToRemove) {
-            return new AccountState(accId, txId, Sets.difference(coins, coinsToRemove).immutableCopy());
+        public AccountState removeCoins(IgniteUuid txId, Set<Integer> coinsToRmv) {
+            return new AccountState(accId, txId, Sets.difference(coins, coinsToRmv).immutableCopy());
         }
 
         /** {@inheritDoc} */
@@ -418,11 +415,11 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
     /**
      * @param coinsNum Coins number.
      */
-    private Set<Integer> generateCoins(AtomicInteger coinsCounter, int coinsNum) {
+    private Set<Integer> generateCoins(AtomicInteger coinsCntr, int coinsNum) {
         Set<Integer> res = new HashSet<>();
 
         for (int i = 0; i < coinsNum; i++)
-            res.add(coinsCounter.incrementAndGet());
+            res.add(coinsCntr.incrementAndGet());
 
         return res;
     }
@@ -479,23 +476,35 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
     private class TransferAmountTxThread extends Thread {
         /** */
         private CountDownLatch firstTransactionLatch;
+
         /** */
         private IgniteEx ignite;
+
         /** */
         private String cacheName;
+
         /** */
-        private int txIndex;
+        private int workerIdx;
+
         /** */
         private Random random = new Random();
 
+        /** */
+        private final boolean colocatedAccounts;
+
         /**
          * @param ignite Ignite.
          */
-        private TransferAmountTxThread(CountDownLatch firstTransactionLatch, final IgniteEx ignite, String cacheName, int txIndex) {
+        private TransferAmountTxThread(CountDownLatch firstTransactionLatch,
+            final IgniteEx ignite,
+            String cacheName,
+            int workerIdx,
+            boolean colocatedAccounts) {
             this.firstTransactionLatch = firstTransactionLatch;
             this.ignite = ignite;
             this.cacheName = cacheName;
-            this.txIndex = txIndex;
+            this.workerIdx = workerIdx;
+            this.colocatedAccounts = colocatedAccounts;
         }
 
         /** {@inheritDoc} */
@@ -514,7 +523,6 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
         /**
          * @throws IgniteException if fails
          */
-        @SuppressWarnings("unchecked")
         private void updateInTransaction(IgniteCache<Integer, AccountState> cache) throws IgniteException {
             int accIdFrom;
             int accIdTo;
@@ -526,11 +534,16 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
                 if (accIdFrom == accIdTo)
                     continue;
 
-                ClusterNode primaryForAccFrom = ignite.cachex(cacheName).affinity().mapKeyToNode(accIdFrom);
-                ClusterNode primaryForAccTo = ignite.cachex(cacheName).affinity().mapKeyToNode(accIdTo);
+                Affinity<Object> affinity = ignite.affinity(cacheName);
+
+                ClusterNode primaryForAccFrom = affinity.mapKeyToNode(accIdFrom);
+                assertNotNull(primaryForAccFrom);
+
+                ClusterNode primaryForAccTo = affinity.mapKeyToNode(accIdTo);
+                assertNotNull(primaryForAccTo);
 
                 // Allows only transaction between accounts that primary on the same node if corresponding flag is enabled.
-                if (!crossNodeTransactions() && !primaryForAccFrom.id().equals(primaryForAccTo.id()))
+                if (colocatedAccounts && !primaryForAccFrom.id().equals(primaryForAccTo.id()))
                     continue;
 
                 break;
@@ -541,7 +554,10 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
             try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                 acctFrom = cache.get(accIdFrom);
+                assertNotNull(acctFrom);
+
                 acctTo = cache.get(accIdTo);
+                assertNotNull(acctTo);
 
                 Set<Integer> coinsToTransfer = acctFrom.coinsToTransfer(random);
 
@@ -553,23 +569,8 @@ public class AbstractTransactionIntergrityTest extends GridCommonAbstractTest {
 
                 tx.commit();
 
-                completedTxs[txIndex].put(tx.xid(), new TxState(acctFrom, acctTo, nextFrom, nextTo, coinsToTransfer));
-            }
-        }
-
-        /**
-         * @param curr current
-         * @return random value
-         */
-        private long getNextAccountId(long curr) {
-            long randomVal;
-
-            do {
-                randomVal = random.nextInt(accountsCount());
+                completedTxs[workerIdx].put(tx.xid(), new TxState(acctFrom, acctTo, nextFrom, nextTo, coinsToTransfer));
             }
-            while (curr == randomVal);
-
-            return randomVal;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
index 3260607..473eaf5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
@@ -17,20 +17,26 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
+import java.util.Collection;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.function.BiFunction;
+import java.util.function.Consumer;
 import java.util.function.Supplier;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.cache.tree.SearchRow;
 import org.apache.ignite.testframework.GridTestUtils;
 
+import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
+
 /**
  * Test cases that check transaction data integrity after transaction commit failed.
  */
@@ -45,81 +51,96 @@ public class TransactionIntegrityWithPrimaryIndexCorruptionTest extends Abstract
         super.afterTest();
     }
 
-    /** {@inheritDoc} */
-    @Override protected long getTestTimeout() {
-        return 60 * 1000L;
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitPrimaryColocatedThrowsError() throws Exception {
+        doTestTransferAmount0(true, true, () -> new AssertionError("Test"));
     }
 
-    /**
-     * Throws a test {@link AssertionError} during tx commit from {@link BPlusTree} and checks after that data is consistent.
-     */
-    public void testPrimaryIndexCorruptionDuringCommitOnPrimaryNode1() throws Exception {
-        doTestTransferAmount(new IndexCorruptionFailoverScenario(
-            true,
-            (hnd, tree) -> hnd instanceof BPlusTree.Search,
-            failoverPredicate(true, () -> new AssertionError("Test")))
-        );
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitPrimaryColocatedThrowsUnchecked() throws Exception {
+        doTestTransferAmount0(true, true, () -> new RuntimeException("Test"));
     }
 
-    /**
-     * Throws a test {@link RuntimeException} during tx commit from {@link BPlusTree} and checks after that data is consistent.
-     */
-    public void testPrimaryIndexCorruptionDuringCommitOnPrimaryNode2() throws Exception {
-        doTestTransferAmount(new IndexCorruptionFailoverScenario(
-            true,
-            (hnd, tree) -> hnd instanceof BPlusTree.Search,
-            failoverPredicate(true, () -> new RuntimeException("Test")))
-        );
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitPrimaryColocatedThrowsChecked() throws Exception {
+        doTestTransferAmount0(true, true, () -> new IgniteCheckedException("Test"));
     }
 
-    /**
-     * Throws a test {@link AssertionError} during tx commit from {@link BPlusTree} and checks after that data is consistent.
-     */
-    public void testPrimaryIndexCorruptionDuringCommitOnBackupNode() throws Exception {
-        doTestTransferAmount(new IndexCorruptionFailoverScenario(
-            true,
-            (hnd, tree) -> hnd instanceof BPlusTree.Search,
-            failoverPredicate(false, () -> new AssertionError("Test")))
-        );
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitPrimaryNonColocatedThrowsError() throws Exception {
+        doTestTransferAmount0(false, true, () -> new AssertionError("Test"));
     }
 
-    /**
-     * Throws a test {@link IgniteCheckedException} during tx commit from {@link BPlusTree} and checks after that data is consistent.
-     */
-    public void testPrimaryIndexCorruptionDuringCommitOnPrimaryNode3() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9082");
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitPrimaryNonColocatedThrowsUnchecked() throws Exception {
+        doTestTransferAmount0(false, true, () -> new RuntimeException("Test"));
+    }
 
-        doTestTransferAmount(new IndexCorruptionFailoverScenario(
-            false,
-            (hnd, tree) -> hnd instanceof BPlusTree.Search,
-            failoverPredicate(true, () -> new IgniteCheckedException("Test")))
-        );
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitPrimaryNonColocatedThrowsChecked() throws Exception {
+        doTestTransferAmount0(false, true, () -> new IgniteCheckedException("Test"));
+    }
+
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitBackupColocatedThrowsError() throws Exception {
+        doTestTransferAmount0(true, false, () -> new AssertionError("Test"));
+    }
+
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitBackupColocatedThrowsUnchecked() throws Exception {
+        doTestTransferAmount0(true, false, () -> new RuntimeException("Test"));
+    }
+
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitBackupColocatedThrowsChecked() throws Exception {
+        doTestTransferAmount0(true, false, () -> new IgniteCheckedException("Test"));
+    }
+
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitBackupNonColocatedThrowsError() throws Exception {
+        doTestTransferAmount0(false, false, () -> new AssertionError("Test"));
+    }
+
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitBackupNonColocatedThrowsUnchecked() throws Exception {
+        doTestTransferAmount0(false, false, () -> new RuntimeException("Test"));
+    }
+
+    /** */
+    public void testPrimaryIndexCorruptionDuringCommitBackupNonColocatedThrowsChecked() throws Exception {
+        doTestTransferAmount0(false, false, () -> new IgniteCheckedException("Test"));
     }
 
     /**
      * Creates failover predicate which generates error during transaction commmit.
      *
-     * @param failOnPrimary If {@code true} index should be failed on transaction primary node.
+     * @param failOnPrimary If {@code true} index should be failed on transaction primary node, otherwise on backup.
      * @param errorSupplier Supplier to create various errors.
+     * @param errorConsumer Consumer to track unexpected errors while committing.
      */
     private BiFunction<IgniteEx, SearchRow, Throwable> failoverPredicate(
         boolean failOnPrimary,
-        Supplier<Throwable> errorSupplier
+        Supplier<Throwable> errorSupplier,
+        Consumer<Throwable> errorConsumer
     ) {
         return (ignite, row) -> {
-            int cacheId = row.cacheId();
-            int partId = row.key().partition();
-
-            final ClusterNode locNode = ignite.localNode();
-            final AffinityTopologyVersion curTopVer = ignite.context().discovery().topologyVersionEx();
-
-            // Throw exception if current node is primary for given row.
-            return ignite.cachesx(c -> c.context().cacheId() == cacheId)
-                .stream()
-                .filter(c -> c.context().affinity().primaryByPartition(locNode, partId, curTopVer) == failOnPrimary)
-                .map(c -> errorSupplier.get())
-                .findFirst()
-                .orElse(null);
+            try {
+                int cacheId = row.cacheId();
+                int partId = row.key().partition();
+
+                GridDhtPartitionTopology top = ignite.context().cache().cacheGroup(cacheId).topology();
+
+                GridDhtLocalPartition part = top.localPartition(partId);
+
+                assertTrue("Illegal partition state for mapped tx: " + part, part != null && part.state() == OWNING);
+
+                return part.primary(top.readyTopologyVersion()) == failOnPrimary ? errorSupplier.get() : null;
+            }
+            catch (Throwable e) {
+                errorConsumer.accept(e);
+
+                throw e;
+            }
         };
     }
 
@@ -130,68 +151,68 @@ public class TransactionIntegrityWithPrimaryIndexCorruptionTest extends Abstract
         /** Failed node index. */
         static final int failedNodeIdx = 1;
 
-        /** Is node stopping expected after failover. */
-        private final boolean nodeStoppingExpected;
-
-        /** Predicate that will choose an instance of {@link BPlusTree} and page operation
-         * to make further failover in this tree using {@link #failoverPredicate}. */
-        private final BiFunction<PageHandler, BPlusTree, Boolean> treeCorruptionPredicate;
+        /**
+         * Predicate that will choose an instance of {@link BPlusTree} and page operation to make further failover in
+         * this tree using {@link #failoverPred}.
+         */
+        private final BiFunction<PageHandler, BPlusTree, Boolean> treeCorruptionPred;
 
         /** Function that may return error during row insertion into {@link BPlusTree}. */
-        private final BiFunction<IgniteEx, SearchRow, Throwable> failoverPredicate;
+        private final BiFunction<IgniteEx, SearchRow, Throwable> failoverPred;
 
         /**
-         * @param nodeStoppingExpected Node stopping expected.
-         * @param treeCorruptionPredicate Tree corruption predicate.
-         * @param failoverPredicate Failover predicate.
+         * @param treeCorruptionPred Tree corruption predicate.
+         * @param failoverPred Failover predicate.
          */
         IndexCorruptionFailoverScenario(
-            boolean nodeStoppingExpected,
-            BiFunction<PageHandler, BPlusTree, Boolean> treeCorruptionPredicate,
-            BiFunction<IgniteEx, SearchRow, Throwable> failoverPredicate
+            BiFunction<PageHandler, BPlusTree, Boolean> treeCorruptionPred,
+            BiFunction<IgniteEx, SearchRow, Throwable> failoverPred
         ) {
-            this.nodeStoppingExpected = nodeStoppingExpected;
-            this.treeCorruptionPredicate = treeCorruptionPredicate;
-            this.failoverPredicate = failoverPredicate;
+            this.treeCorruptionPred = treeCorruptionPred;
+            this.failoverPred = failoverPred;
         }
 
         /** {@inheritDoc} */
         @Override public void beforeNodesStarted() {
             BPlusTree.pageHndWrapper = (tree, hnd) -> {
-                final IgniteEx locIgnite = (IgniteEx) Ignition.localIgnite();
+                final IgniteEx locIgnite = (IgniteEx)Ignition.localIgnite();
 
-                if (!locIgnite.name().endsWith(String.valueOf(failedNodeIdx)))
+                if (getTestIgniteInstanceIndex(locIgnite.name()) != failedNodeIdx)
                     return hnd;
 
-                if (treeCorruptionPredicate.apply(hnd, tree)) {
-                    log.info("Created corrupted tree handler for -> " + hnd + " " + tree);
+                if (treeCorruptionPred.apply(hnd, tree)) {
+                    log.info("Created corrupted tree handler [nodeOrder=" + locIgnite.localNode().order() + ", hnd=" + hnd +
+                        ", tree=" + tree + ']');
 
-                    PageHandler<Object, BPlusTree.Result> delegate = (PageHandler<Object, BPlusTree.Result>) hnd;
+                    PageHandler<Object, BPlusTree.Result> delegate = (PageHandler<Object, BPlusTree.Result>)hnd;
 
                     return new PageHandler<BPlusTree.Get, BPlusTree.Result>() {
-                        @Override public BPlusTree.Result run(int cacheId, long pageId, long page, long pageAddr, PageIO io, Boolean walPlc, BPlusTree.Get arg, int lvl) throws IgniteCheckedException {
-                            log.info("Invoked " + " " + cacheId + " " + arg.toString() + " for BTree (" + corruptionEnabled + ") -> " + arg.row() + " / " + arg.row().getClass());
+                        @Override public BPlusTree.Result run(int cacheId, long pageId, long page, long pageAddr, PageIO io,
+                            Boolean walPlc, BPlusTree.Get arg, int lvl) throws IgniteCheckedException {
+                            log.info("Invoked [cachedId=" + cacheId + ", hnd=" + arg.toString() +
+                                ", corruption=" + corruptionEnabled + ", row=" + arg.row() + ", rowCls=" + arg.row().getClass() + ']');
 
                             if (corruptionEnabled && (arg.row() instanceof SearchRow)) {
-                                SearchRow row = (SearchRow) arg.row();
+                                SearchRow row = (SearchRow)arg.row();
 
                                 // Store cacheId to search row explicitly, as it can be zero if there is one cache in a group.
-                                Throwable res = failoverPredicate.apply(locIgnite, new SearchRow(cacheId, row.key()));
+                                Throwable res = failoverPred.apply(locIgnite, new SearchRow(cacheId, row.key()));
 
                                 if (res != null) {
                                     if (res instanceof Error)
-                                        throw (Error) res;
+                                        throw (Error)res;
                                     else if (res instanceof RuntimeException)
-                                        throw (RuntimeException) res;
+                                        throw (RuntimeException)res;
                                     else if (res instanceof IgniteCheckedException)
-                                        throw (IgniteCheckedException) res;
+                                        throw (IgniteCheckedException)res;
                                 }
                             }
 
                             return delegate.run(cacheId, pageId, page, pageAddr, io, walPlc, arg, lvl);
                         }
 
-                        @Override public boolean releaseAfterWrite(int cacheId, long pageId, long page, long pageAddr, BPlusTree.Get g, int lvl) {
+                        @Override public boolean releaseAfterWrite(int cacheId, long pageId, long page, long pageAddr,
+                            BPlusTree.Get g, int lvl) {
                             return g.canRelease(pageId, lvl);
                         }
                     };
@@ -212,27 +233,68 @@ public class TransactionIntegrityWithPrimaryIndexCorruptionTest extends Abstract
             // Disable index corruption.
             BPlusTree.pageHndWrapper = (tree, hnd) -> hnd;
 
-            if (nodeStoppingExpected) {
-                // Wait until node with corrupted index will left cluster.
-                GridTestUtils.waitForCondition(() -> {
-                    try {
-                        grid(failedNodeIdx);
-                    }
-                    catch (IgniteIllegalStateException e) {
-                        return true;
-                    }
+            // Wait until node with corrupted index will left cluster.
+            GridTestUtils.waitForCondition(() -> {
+                try {
+                    grid(failedNodeIdx);
+                }
+                catch (IgniteIllegalStateException e) {
+                    return true;
+                }
 
-                    return false;
-                }, getTestTimeout());
+                return false;
+            }, getTestTimeout());
 
-                // Failed node should be stopped.
-                GridTestUtils.assertThrows(log, () -> grid(failedNodeIdx), IgniteIllegalStateException.class, "");
+            // Failed node should be stopped.
+            GridTestUtils.assertThrows(log, () -> grid(failedNodeIdx), IgniteIllegalStateException.class, null);
 
-                // Re-start failed node.
-                startGrid(failedNodeIdx);
+            // Re-start failed node.
+            startGrid(failedNodeIdx);
 
-                awaitPartitionMapExchange();
-            }
+            awaitPartitionMapExchange();
+        }
+    }
+
+    /**
+     * Test transfer amount with extended error recording.
+     *
+     * @param colocatedAccount Colocated account.
+     * @param failOnPrimary {@code True} if fail on primary, else on backup.
+     * @param supplier Fail reason supplier.
+     * @throws Exception If failover predicate execution is failed.
+     */
+    private void doTestTransferAmount0(boolean colocatedAccount, boolean failOnPrimary, Supplier<Throwable> supplier) throws Exception {
+        ErrorTracker errTracker = new ErrorTracker();
+
+        doTestTransferAmount(
+            new IndexCorruptionFailoverScenario(
+                (hnd, tree) -> hnd instanceof BPlusTree.Search,
+                failoverPredicate(failOnPrimary, supplier, errTracker)),
+            colocatedAccount
+        );
+
+        for (Throwable throwable : errTracker.errors())
+            log.error("Recorded error", throwable);
+
+        if (!errTracker.errors().isEmpty())
+            fail("Test run has error");
+    }
+
+    /** */
+    private static class ErrorTracker implements Consumer<Throwable> {
+        /** Queue. */
+        private final Queue<Throwable> q = new ConcurrentLinkedQueue<>();
+
+        /** {@inheritDoc} */
+        @Override public void accept(Throwable throwable) {
+            q.add(throwable);
+        }
+
+        /**
+         * @return Recorded errors.
+         */
+        public Collection<Throwable> errors() {
+            return q;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithSystemWorkerDeathTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithSystemWorkerDeathTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithSystemWorkerDeathTest.java
index 25aae4b..551335f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithSystemWorkerDeathTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithSystemWorkerDeathTest.java
@@ -41,9 +41,7 @@ public class TransactionIntegrityWithSystemWorkerDeathTest extends AbstractTrans
         return false;
     }
 
-    /**
-     *
-     */
+    /** */
     public void testFailoverWithDiscoWorkerTermination() throws Exception {
         doTestTransferAmount(new FailoverScenario() {
             static final int failedNodeIdx = 1;
@@ -83,7 +81,7 @@ public class TransactionIntegrityWithSystemWorkerDeathTest extends AbstractTrans
 
                 awaitPartitionMapExchange();
             }
-        });
+        }, true);
     }
 
     /**


[22/50] [abbrv] ignite git commit: IGNITE-5935: MVCC TX: Tx recovery protocol

Posted by sb...@apache.org.
IGNITE-5935: MVCC TX: Tx recovery protocol

This closes #4920


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5939a947
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5939a947
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5939a947

Branch: refs/heads/ignite-gg-14206
Commit: 5939a94763c8a3e92b66b3f591a816dd6c49f35a
Parents: 82d2efe
Author: ipavlukhin <vo...@gmail.com>
Authored: Fri Oct 19 17:40:12 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Fri Oct 19 17:40:12 2018 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |   6 +-
 .../communication/GridIoMessageFactory.java     |  18 +
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../cache/IgniteCacheOffheapManager.java        |   6 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |   5 +
 .../cache/PartitionUpdateCounter.java           |  30 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |  11 -
 .../GridDistributedTxRemoteAdapter.java         |  52 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   4 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  38 --
 .../distributed/dht/GridDhtTxPrepareFuture.java |   2 +-
 .../topology/GridClientPartitionTopology.java   |   3 +-
 .../dht/topology/GridDhtLocalPartition.java     |   7 +
 .../dht/topology/GridDhtPartitionTopology.java  |   5 +-
 .../topology/GridDhtPartitionTopologyImpl.java  |   6 +-
 .../processors/cache/mvcc/MvccProcessor.java    |   7 -
 .../cache/mvcc/MvccProcessorImpl.java           | 217 +++++-
 .../processors/cache/mvcc/MvccUtils.java        |   6 +-
 .../mvcc/msg/MvccRecoveryFinishedMessage.java   | 116 ++++
 .../PartitionCountersNeighborcastRequest.java   | 145 ++++
 .../PartitionCountersNeighborcastResponse.java  | 114 ++++
 .../persistence/GridCacheOffheapManager.java    |  13 +
 .../cache/transactions/IgniteTxAdapter.java     |   3 +-
 .../cache/transactions/IgniteTxHandler.java     | 125 ++++
 .../cache/transactions/IgniteTxManager.java     | 123 +++-
 .../PartitionCountersNeighborcastFuture.java    | 211 ++++++
 .../cache/transactions/TxCounters.java          |   3 +-
 .../continuous/GridContinuousProcessor.java     |   6 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   2 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |  69 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 +-
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |   2 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java | 133 +++-
 ...licatedTxOriginatingNodeFailureSelfTest.java |   2 +-
 .../cache/mvcc/CacheMvccTransactionsTest.java   |   4 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |   2 +-
 .../IgniteCacheTxRecoverySelfTestSuite.java     |   2 +-
 .../cache/mvcc/CacheMvccTxRecoveryTest.java     | 654 +++++++++++++++++++
 ...GridIndexRebuildWithMvccEnabledSelfTest.java |   3 +-
 .../testsuites/IgniteCacheMvccSqlTestSuite.java |  58 ++
 40 files changed, 1980 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 2599d7a..7492e51 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -43,9 +43,7 @@ import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridInvokeValue;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxEnlistRequest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxEnlistResponse;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccRecoveryFinishedMessage;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -171,7 +169,7 @@ public class MessageCodeGenerator {
 
 //        gen.generateAll(true);
 
-        gen.generateAndWrite(GridNearTxEnlistResponse.class);
+        gen.generateAndWrite(MvccRecoveryFinishedMessage.class);
 
 //        gen.generateAndWrite(GridNearAtomicUpdateRequest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index e405d7d..3f4eb18 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -134,9 +134,12 @@ import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTxAndQ
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccActiveQueriesMessage;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccFutureResponse;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccQuerySnapshotRequest;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccRecoveryFinishedMessage;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccTxSnapshotRequest;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccWaitTxsRequest;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastRequest;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastResponse;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
@@ -1096,6 +1099,21 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
+            case 164:
+                msg = new MvccRecoveryFinishedMessage();
+
+                break;
+
+            case 165:
+                msg = new PartitionCountersNeighborcastRequest();
+
+                break;
+
+            case 166:
+                msg = new PartitionCountersNeighborcastResponse();
+
+                break;
+
                 // [-3..119] [124..129] [-23..-27] [-36..-55]- this
             // [120..123] - DR
             // [-4..-22, -30..-35] - SQL

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 6af9678..0b8dd75 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
@@ -1405,7 +1405,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     grp.affinity().similarAffinityKey());
 
                 if (sndCounters) {
-                    CachePartitionPartialCountersMap cntrsMap = grp.topology().localUpdateCounters(true);
+                    CachePartitionPartialCountersMap cntrsMap = grp.topology().localUpdateCounters(true, true);
 
                     m.addPartitionUpdateCounters(grp.groupId(),
                         newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap));
@@ -1429,7 +1429,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 top.similarAffinityKey());
 
             if (sndCounters) {
-                CachePartitionPartialCountersMap cntrsMap = top.localUpdateCounters(true);
+                CachePartitionPartialCountersMap cntrsMap = top.localUpdateCounters(true, true);
 
                 m.addPartitionUpdateCounters(top.groupId(),
                     newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap));

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index 044830c..e9ec025 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -1067,8 +1067,12 @@ public interface IgniteCacheOffheapManager {
          * Return PendingTree for data store.
          *
          * @return PendingTree instance.
-         * @throws IgniteCheckedException
          */
         PendingEntriesTree pendingTree();
+
+        /**
+         * Flushes pending update counters closing all possible gaps.
+         */
+        void finalizeUpdateCountres();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index e40cc53..e547784 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -1556,6 +1556,11 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         }
 
         /** {@inheritDoc} */
+        @Override public void finalizeUpdateCountres() {
+            pCntr.finalizeUpdateCountres();
+        }
+
+        /** {@inheritDoc} */
         @Override public String name() {
             return name;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
index b5960ab..fe44708 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java
@@ -17,8 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.PriorityQueue;
-import java.util.Queue;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteLogger;
 import org.jetbrains.annotations.NotNull;
@@ -31,7 +30,7 @@ public class PartitionUpdateCounter {
     private IgniteLogger log;
 
     /** Queue of counter update tasks*/
-    private final Queue<Item> queue = new PriorityQueue<>();
+    private final TreeSet<Item> queue = new TreeSet<>();
 
     /** Counter. */
     private final AtomicLong cntr = new AtomicLong();
@@ -161,21 +160,34 @@ public class PartitionUpdateCounter {
      * @return Retrieves the minimum update counter task from queue.
      */
     private Item poll() {
-        return queue.poll();
+        return queue.pollFirst();
     }
 
     /**
      * @return Checks the minimum update counter task from queue.
      */
     private Item peek() {
-        return queue.peek();
+        return queue.isEmpty() ? null : queue.first();
+
     }
 
     /**
      * @param item Adds update task to priority queue.
      */
     private void offer(Item item) {
-        queue.offer(item);
+        queue.add(item);
+    }
+
+    /**
+     * Flushes pending update counters closing all possible gaps.
+     */
+    public synchronized void finalizeUpdateCountres() {
+        Item last = queue.pollLast();
+
+        if (last != null)
+            update(last.start + last.delta);
+
+        queue.clear();
     }
 
     /**
@@ -199,11 +211,7 @@ public class PartitionUpdateCounter {
 
         /** {@inheritDoc} */
         @Override public int compareTo(@NotNull Item o) {
-            int cmp = Long.compare(this.start, o.start);
-
-            assert cmp != 0;
-
-            return cmp;
+            return Long.compare(this.start, o.start);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
index 3fb1e4f..5e0deb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
@@ -146,17 +146,6 @@ public class GridCacheTxRecoveryFuture extends GridCacheCompoundIdentityFuture<B
      */
     @SuppressWarnings("ConstantConditions")
     public void prepare() {
-        if (tx.txState().mvccEnabled()) { // TODO IGNITE-5935
-            U.error(log, "Cannot commit MVCC enabled transaction by recovery procedure. " +
-                "Operation is usupported at the moment [tx=" + CU.txString(tx) + ']');
-
-            onDone(false);
-
-            markInitialized();
-
-            return;
-        }
-
         if (nearTxCheck) {
             UUID nearNodeId = tx.eventNodeId();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 4db4685..3cabaec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -50,9 +50,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWra
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
@@ -770,15 +767,15 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
 
                         // Apply update counters.
                         if (txCntrs != null)
-                            applyPartitionsUpdatesCounters(txCntrs.updateCounters());
+                            cctx.tm().txHandler().applyPartitionsUpdatesCounters(txCntrs.updateCounters());
 
-                            cctx.mvccCaching().onTxFinished(this, true);
+                        cctx.mvccCaching().onTxFinished(this, true);
 
-                            if (!near() && !F.isEmpty(dataEntries) && cctx.wal() != null) {
-                                // Set new update counters for data entries received from persisted tx entries.
-                                List<DataEntry> entriesWithCounters = dataEntries.stream()
-                                        .map(tuple -> tuple.get1().partitionCounter(tuple.get2().updateCounter()))
-                                        .collect(Collectors.toList());
+                        if (!near() && !F.isEmpty(dataEntries) && cctx.wal() != null) {
+                            // Set new update counters for data entries received from persisted tx entries.
+                            List<DataEntry> entriesWithCounters = dataEntries.stream()
+                                .map(tuple -> tuple.get1().partitionCounter(tuple.get2().updateCounter()))
+                                .collect(Collectors.toList());
 
                             cctx.wal().log(new DataRecord(entriesWithCounters));
                         }
@@ -921,7 +918,7 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                 TxCounters counters = txCounters(false);
 
                 if (counters != null)
-                    applyPartitionsUpdatesCounters(counters.updateCounters());
+                    cctx.tm().txHandler().applyPartitionsUpdatesCounters(counters.updateCounters());
 
                 state(ROLLED_BACK);
 
@@ -996,39 +993,6 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
         }
     }
 
-    /**
-     * Applies partition counters updates for mvcc transactions.
-     *
-     * @param counters Counters values to be updated.
-     */
-    private void applyPartitionsUpdatesCounters(Iterable<PartitionUpdateCountersMessage> counters) {
-        if (counters == null)
-            return;
-
-        int cacheId = CU.UNDEFINED_CACHE_ID;
-        GridDhtPartitionTopology top = null;
-
-        for (PartitionUpdateCountersMessage counter : counters) {
-            if (counter.cacheId() != cacheId) {
-                GridCacheContext ctx0 = cctx.cacheContext(cacheId = counter.cacheId());
-
-                assert ctx0.mvccEnabled();
-
-                top = ctx0.topology();
-            }
-
-            assert top != null;
-
-            for (int i = 0; i < counter.size(); i++) {
-                GridDhtLocalPartition part = top.localPartition(counter.partition(i));
-
-                assert part != null;
-
-                part.updateCounter(counter.initialCounter(i), counter.updatesCount(i));
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(GridDistributedTxRemoteAdapter.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
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 9f96b46..d0fbd90 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
@@ -375,7 +375,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
                 false,
                 false,
                 tx.mvccSnapshot(),
-                tx.filterUpdateCountersForBackupNode(n));
+                cctx.tm().txHandler().filterUpdateCountersForBackupNode(tx, n));
 
             try {
                 cctx.io().send(n, req, tx.ioPolicy());
@@ -488,7 +488,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
                 false,
                 false,
                 mvccSnapshot,
-                commit ? null : tx.filterUpdateCountersForBackupNode(n));
+                commit ? null : cctx.tm().txHandler().filterUpdateCountersForBackupNode(tx, n));
 
             req.writeVersion(tx.writeVersion() != null ? tx.writeVersion() : tx.xidVersion());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
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 483990f..86f9c3c 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.io.Externalizable;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -34,7 +33,6 @@ import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
@@ -48,7 +46,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
 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.util.F0;
 import org.apache.ignite.internal.util.GridLeanMap;
@@ -944,41 +941,6 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         return prepFut;
     }
 
-    /**
-     * @param node Backup node.
-     * @return Partition counters map for the given backup node.
-     */
-    public List<PartitionUpdateCountersMessage> filterUpdateCountersForBackupNode(ClusterNode node) {
-        TxCounters txCntrs = txCounters(false);
-
-        if (txCntrs == null || F.isEmpty(txCntrs.updateCounters()))
-            return null;
-
-        Collection<PartitionUpdateCountersMessage> updCntrs = txCntrs.updateCounters();
-
-        List<PartitionUpdateCountersMessage> res = new ArrayList<>(updCntrs.size());
-
-        AffinityTopologyVersion top = topologyVersionSnapshot();
-
-        for (PartitionUpdateCountersMessage partCntrs : updCntrs) {
-            GridCacheAffinityManager affinity = cctx.cacheContext(partCntrs.cacheId()).affinity();
-
-            PartitionUpdateCountersMessage resCntrs = new PartitionUpdateCountersMessage(partCntrs.cacheId(), partCntrs.size());
-
-            for (int i = 0; i < partCntrs.size(); i++) {
-                int part = partCntrs.partition(i);
-
-                if (affinity.backupByPartition(node, part, top))
-                    resCntrs.add(part, partCntrs.initialCounter(i), partCntrs.updatesCount(i));
-            }
-
-            if (resCntrs.size() > 0)
-                res.add(resCntrs);
-        }
-
-        return res;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(GridDhtTxLocalAdapter.class, this, "nearNodes", nearMap.keySet(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
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 c505677..609bff8 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
@@ -1398,7 +1398,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
                 tx.storeWriteThrough(),
                 retVal,
                 mvccSnapshot,
-                tx.filterUpdateCountersForBackupNode(n));
+                cctx.tm().txHandler().filterUpdateCountersForBackupNode(tx, n));
 
             req.queryUpdate(dhtMapping.queryUpdate());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
index 9140322..cd6e254 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java
@@ -1234,7 +1234,8 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros) {
+    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros,
+        boolean finalizeCntrsBeforeCollecting) {
         return CachePartitionPartialCountersMap.EMPTY;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
index 253a56a..2ddc0d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
@@ -1371,6 +1371,13 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
     }
 
     /**
+     * Flushes pending update counters closing all possible gaps.
+     */
+    public void finalizeUpdateCountres() {
+        store.finalizeUpdateCountres();
+    }
+
+    /**
      * Removed entry holder.
      */
     private static class RemovedEntryHolder {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
index b6cb5bb..25b284e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java
@@ -350,9 +350,12 @@ public interface GridDhtPartitionTopology {
     public CachePartitionFullCountersMap fullUpdateCounters();
 
     /**
+     * @param skipZeros {@code True} for adding zero counter to map.
+     * @param finalizeCntrsBeforeCollecting {@code True} indicates that partition counters should be finalized.
      * @return Partition update counters.
      */
-    public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros);
+    public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros,
+        boolean finalizeCntrsBeforeCollecting);
 
     /**
      * @return Partition cache sizes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
index 94bb7f1..1f338d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java
@@ -2657,7 +2657,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros) {
+    @Override public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros,
+        boolean finalizeCntrsBeforeCollecting) {
         lock.readLock().lock();
 
         try {
@@ -2678,6 +2679,9 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 if (part == null)
                     continue;
 
+                if (finalizeCntrsBeforeCollecting)
+                    part.finalizeUpdateCountres();
+
                 long updCntr = part.updateCounter();
                 long initCntr = part.initialUpdateCounter();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
index a09468f..a926acf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
@@ -180,13 +180,6 @@ public interface MvccProcessor extends GridProcessor {
     /**
      * Requests snapshot on Mvcc coordinator.
      *
-     * @return Snapshot future.
-     */
-    IgniteInternalFuture<MvccSnapshot> requestSnapshotAsync();
-
-    /**
-     * Requests snapshot on Mvcc coordinator.
-     *
      * @param tx Transaction.
      * @return Snapshot future.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index f17c137..9fcafb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -20,14 +20,17 @@ package org.apache.ignite.internal.processors.cache.mvcc;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -68,6 +71,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccActiveQueriesMes
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccFutureResponse;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccMessage;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccQuerySnapshotRequest;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccRecoveryFinishedMessage;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccTxSnapshotRequest;
 import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccWaitTxsRequest;
@@ -189,8 +193,11 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     /** */
     private final GridAtomicLong committedCntr = new GridAtomicLong(MVCC_INITIAL_CNTR);
 
-    /** */
-    private final Map<Long, Long> activeTxs = new HashMap<>();
+    /**
+     * Contains active transactions on mvcc coordinator. Key is mvcc counter.
+     * Access is protected by "this" monitor.
+     */
+    private final Map<Long, ActiveTx> activeTxs = new HashMap<>();
 
     /** Active query trackers. */
     private final Map<Long, MvccQueryTracker> activeTrackers = new ConcurrentHashMap<>();
@@ -223,6 +230,11 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     private volatile boolean mvccSupported = true;
 
     /**
+     * Maps failed node id to votes accumulator for that node.
+     */
+    private final ConcurrentHashMap<UUID, RecoveryBallotBox> recoveryBallotBoxes = new ConcurrentHashMap<>();
+
+    /**
      * @param ctx Context.
      */
     public MvccProcessorImpl(GridKernalContext ctx) {
@@ -363,8 +375,12 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
     /** {@inheritDoc} */
     @Override public void onExchangeDone(boolean newCrd, DiscoCache discoCache, Map<UUID, GridLongList> activeQueries) {
-        if (!newCrd)
+        if (!newCrd) {
+            if (curCrd != null && ctx.localNodeId().equals(curCrd.nodeId()) && discoCache != null)
+                cleanupOrphanedServerTransactions(discoCache.serverNodes());
+
             return;
+        }
 
         ctx.cache().context().tm().rollbackMvccTxOnCoordinatorChange();
 
@@ -391,6 +407,33 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         }
     }
 
+    /**
+     * Cleans up active transacitons lost near node which is server. Executed on coordinator.
+     * @param liveSrvs Live server nodes at the moment of cleanup.
+     */
+    private void cleanupOrphanedServerTransactions(Collection<ClusterNode> liveSrvs) {
+        Set<UUID> ids = liveSrvs.stream()
+            .map(ClusterNode::id)
+            .collect(Collectors.toSet());
+
+        List<Long> forRmv = new ArrayList<>();
+
+        synchronized (this) {
+            for (Map.Entry<Long, ActiveTx> entry : activeTxs.entrySet()) {
+                // If node started tx is not known as live then remove such tx from active list
+                ActiveTx activeTx = entry.getValue();
+
+                if (activeTx.getClass() == ActiveServerTx.class && !ids.contains(activeTx.nearNodeId))
+                    forRmv.add(entry.getKey());
+            }
+        }
+
+        for (Long txCntr : forRmv)
+            // Committed counter is increased because it is not known if transaction was committed or not and we must
+            // bump committed counter for committed transaction as it is used in (read-only) query snapshot.
+            onTxDone(txCntr, true);
+    }
+
     /** {@inheritDoc} */
     @Override public void processClientActiveQueries(UUID nodeId, @Nullable GridLongList activeQueries) {
         prevCrdQueries.addNodeActiveQueries(nodeId, activeQueries);
@@ -530,17 +573,12 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         if (!ctx.localNodeId().equals(crd.nodeId()) || !initFut.isDone())
             return null;
         else if (tx != null)
-            return assignTxSnapshot(0L);
+            return assignTxSnapshot(0L, ctx.localNodeId(), false);
         else
             return activeQueries.assignQueryCounter(ctx.localNodeId(), 0L);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<MvccSnapshot> requestSnapshotAsync() {
-        return requestSnapshotAsync((IgniteInternalTx)null);
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<MvccSnapshot> requestSnapshotAsync(IgniteInternalTx tx) {
         MvccSnapshotFuture fut = new MvccSnapshotFuture();
 
@@ -585,7 +623,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                 });
             }
             else if (tx != null)
-                lsnr.onResponse(assignTxSnapshot(0L));
+                lsnr.onResponse(assignTxSnapshot(0L, ctx.localNodeId(), false));
             else
                 lsnr.onResponse(activeQueries.assignQueryCounter(ctx.localNodeId(), 0L));
 
@@ -741,9 +779,8 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                 first = false;
             }
 
-            for (MvccSnapshotResponseListener lsnr : map.values()) {
+            for (MvccSnapshotResponseListener lsnr : map.values())
                 U.warn(log, ">>> " + lsnr.toString());
-            }
         }
 
         first = true;
@@ -909,10 +946,8 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         return activeQryTrackers;
     }
 
-    /**
-     * @return Counter.
-     */
-    private MvccSnapshotResponse assignTxSnapshot(long futId) {
+    /** */
+    private MvccSnapshotResponse assignTxSnapshot(long futId, UUID nearId, boolean client) {
         assert initFut.isDone();
         assert crdVer != 0;
         assert ctx.localNodeId().equals(currentCoordinatorId());
@@ -926,14 +961,16 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
             tracking = ver;
             cleanup = committedCntr.get() + 1;
 
-            for (Map.Entry<Long, Long> txVer : activeTxs.entrySet()) {
-                cleanup = Math.min(txVer.getValue(), cleanup);
-                tracking = Math.min(txVer.getKey(), tracking);
+            for (Map.Entry<Long, ActiveTx> entry : activeTxs.entrySet()) {
+                cleanup = Math.min(entry.getValue().tracking, cleanup);
+                tracking = Math.min(entry.getKey(), tracking);
 
-                res.addTx(txVer.getKey());
+                res.addTx(entry.getKey());
             }
 
-            boolean add = activeTxs.put(ver, tracking) == null;
+            ActiveTx activeTx = client ? new ActiveTx(tracking, nearId) : new ActiveServerTx(tracking, nearId);
+
+            boolean add = activeTxs.put(ver, activeTx) == null;
 
             assert add : ver;
         }
@@ -950,10 +987,8 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         return res;
     }
 
-    /**
-     * @param txCntr Counter assigned to transaction.
-     */
-    private void onTxDone(Long txCntr, boolean committed) {
+    /** */
+    private void onTxDone(Long txCntr, boolean increaseCommittedCntr) {
         assert initFut.isDone();
 
         GridFutureAdapter fut;
@@ -961,7 +996,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         synchronized (this) {
             activeTxs.remove(txCntr);
 
-            if (committed)
+            if (increaseCommittedCntr)
                 committedCntr.setIfGreater(txCntr);
         }
 
@@ -1352,10 +1387,14 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
             return;
         }
 
-        MvccSnapshotResponse res = assignTxSnapshot(msg.futureId());
+        MvccSnapshotResponse res = assignTxSnapshot(msg.futureId(), nodeId, node.isClient());
+
+        boolean finishFailed = true;
 
         try {
             sendMessage(node.id(), res);
+
+            finishFailed = false;
         }
         catch (ClusterTopologyCheckedException e) {
             if (log.isDebugEnabled())
@@ -1364,6 +1403,9 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         catch (IgniteCheckedException e) {
             U.error(log, "Failed to send tx snapshot response [msg=" + msg + ", node=" + nodeId + ']', e);
         }
+
+        if (finishFailed)
+            onTxDone(res.counter(), false);
     }
 
     /**
@@ -1390,9 +1432,9 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                 log.debug("Failed to send query counter response, node left [msg=" + msg + ", node=" + nodeId + ']');
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send query counter response [msg=" + msg + ", node=" + nodeId + ']', e);
-
             onQueryDone(nodeId, res.tracking());
+
+            U.error(log, "Failed to send query counter response [msg=" + msg + ", node=" + nodeId + ']', e);
         }
     }
 
@@ -1713,6 +1755,23 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
             activeQueries.onNodeFailed(nodeId);
 
             prevCrdQueries.onNodeFailed(nodeId);
+
+            recoveryBallotBoxes.forEach((nearNodeId, ballotBox) -> {
+                // Put synthetic vote from another failed node
+                ballotBox.vote(nodeId);
+
+                tryFinishRecoveryVoting(nearNodeId, ballotBox);
+            });
+
+            if (discoEvt.eventNode().isClient()) {
+                RecoveryBallotBox ballotBox = recoveryBallotBoxes
+                    .computeIfAbsent(nodeId, uuid -> new RecoveryBallotBox());
+
+                ballotBox
+                    .voters(discoEvt.topologyNodes().stream().map(ClusterNode::id).collect(Collectors.toList()));
+
+                tryFinishRecoveryVoting(nodeId, ballotBox);
+            }
         }
 
         /** {@inheritDoc} */
@@ -1767,6 +1826,8 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                 processNewCoordinatorQueryAckRequest(nodeId, (MvccAckRequestQueryId)msg);
             else if (msg instanceof MvccActiveQueriesMessage)
                 processCoordinatorActiveQueriesMessage(nodeId, (MvccActiveQueriesMessage)msg);
+            else if (msg instanceof MvccRecoveryFinishedMessage)
+                processRecoveryFinishedMessage(nodeId, ((MvccRecoveryFinishedMessage)msg));
             else
                 U.warn(log, "Unexpected message received [node=" + nodeId + ", msg=" + msg + ']');
         }
@@ -1777,6 +1838,82 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         }
     }
 
+    /**
+     * Accumulates transaction recovery votes for a node left the cluster.
+     * Transactions started by the left node are considered not active
+     * when each cluster server node aknowledges that is has finished transactions for the left node.
+     */
+    private static class RecoveryBallotBox {
+        /** */
+        private List<UUID> voters;
+        /** */
+        private final Set<UUID> ballots = new HashSet<>();
+
+        /**
+         * @param voters Nodes which can have transaction started by the left node.
+         */
+        private synchronized void voters(List<UUID> voters) {
+            this.voters = voters;
+        }
+
+        /**
+         * @param nodeId Voting node id.
+         *
+         */
+        private synchronized void vote(UUID nodeId) {
+            ballots.add(nodeId);
+        }
+
+        /**
+         * @return {@code True} if all nodes expected to vote done it.
+         */
+        private synchronized boolean isVotingDone() {
+            if (voters == null)
+                return false;
+
+            return ballots.containsAll(voters);
+        }
+    }
+
+    /**
+     * Process message that one node has finished with transactions for the left node.
+     * @param nodeId Node sent the message.
+     * @param msg Message.
+     */
+    private void processRecoveryFinishedMessage(UUID nodeId, MvccRecoveryFinishedMessage msg) {
+        UUID nearNodeId = msg.nearNodeId();
+
+        RecoveryBallotBox ballotBox = recoveryBallotBoxes.computeIfAbsent(nearNodeId, uuid -> new RecoveryBallotBox());
+
+        ballotBox.vote(nodeId);
+
+        tryFinishRecoveryVoting(nearNodeId, ballotBox);
+    }
+
+    /**
+     * Finishes recovery on coordinator by removing transactions started by the left node
+     * @param nearNodeId Left node.
+     * @param ballotBox Votes accumulator for the left node.
+     */
+    private void tryFinishRecoveryVoting(UUID nearNodeId, RecoveryBallotBox ballotBox) {
+        if (ballotBox.isVotingDone()) {
+            List<Long> recoveredTxs;
+
+            synchronized (this) {
+                recoveredTxs = activeTxs.entrySet().stream()
+                    .filter(e -> e.getValue().nearNodeId.equals(nearNodeId))
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toList());
+            }
+
+            // Committed counter is increased because it is not known if transaction was committed or not and we must
+            // bump committed counter for committed transaction as it is used in (read-only) query snapshot.
+            recoveredTxs.forEach(txCntr -> onTxDone(txCntr, true));
+
+            recoveryBallotBoxes.remove(nearNodeId);
+        }
+    }
+
     /** */
     private interface Waiter {
         /**
@@ -2324,4 +2461,26 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
             }
         }
     }
+
+    /** */
+    private static class ActiveTx {
+        /** */
+        private final long tracking;
+        /** */
+        private final UUID nearNodeId;
+
+        /** */
+        private ActiveTx(long tracking, UUID nearNodeId) {
+            this.tracking = tracking;
+            this.nearNodeId = nearNodeId;
+        }
+    }
+
+    /** */
+    private static class ActiveServerTx extends ActiveTx {
+        /** */
+        private ActiveServerTx(long tracking, UUID nearNodeId) {
+            super(tracking, nearNodeId);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
----------------------------------------------------------------------
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 9441c17..972d4d9 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
@@ -242,7 +242,11 @@ public class MvccUtils {
         if (mvccCntr > snapshotCntr) // we don't see future updates
             return false;
 
-        if (mvccCntr == snapshotCntr) {
+        // Basically we can make fast decision about visibility if found rows from the same transaction.
+        // But we can't make such decision for read-only queries,
+        // because read-only queries use last committed version in it's snapshot which could be actually aborted
+        // (during transaction recovery we do not know whether recovered transaction was committed or aborted).
+        if (mvccCntr == snapshotCntr && snapshotOpCntr != MVCC_READ_OP_CNTR) {
             assert opCntr <= snapshotOpCntr : "rowVer=" + mvccVersion(mvccCrd, mvccCntr, opCntr) + ", snapshot=" + snapshot;
 
             return opCntr < snapshotOpCntr; // we don't see own pending updates

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccRecoveryFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccRecoveryFinishedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccRecoveryFinishedMessage.java
new file mode 100644
index 0000000..a4ea103
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccRecoveryFinishedMessage.java
@@ -0,0 +1,116 @@
+/*
+ * 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.cache.mvcc.msg;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/** */
+public class MvccRecoveryFinishedMessage implements MvccMessage {
+    /** */
+    private static final long serialVersionUID = -505062368078979867L;
+
+    /** */
+    private UUID nearNodeId;
+
+    /** */
+    public MvccRecoveryFinishedMessage() {
+    }
+
+    /** */
+    public MvccRecoveryFinishedMessage(UUID nearNodeId) {
+        this.nearNodeId = nearNodeId;
+    }
+
+    /**
+     * @return Left node id for which transactions were recovered.
+     */
+    public UUID nearNodeId() {
+        return nearNodeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean waitForCoordinatorInit() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean processedFromNioThread() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeUuid("nearNodeId", nearNodeId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                nearNodeId = reader.readUuid("nearNodeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(MvccRecoveryFinishedMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short directType() {
+        return 164;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
new file mode 100644
index 0000000..ffd9a67
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastRequest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.cache.mvcc.msg;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.processors.cache.GridCacheIdMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/** */
+public class PartitionCountersNeighborcastRequest extends GridCacheIdMessage {
+    /** */
+    private static final long serialVersionUID = -1893577108462486998L;
+
+    /** */
+    @GridDirectCollection(PartitionUpdateCountersMessage.class)
+    private Collection<PartitionUpdateCountersMessage> updCntrs;
+
+    /** */
+    private IgniteUuid futId;
+
+    /** */
+    public PartitionCountersNeighborcastRequest() {
+    }
+
+    /** */
+    public PartitionCountersNeighborcastRequest(
+        Collection<PartitionUpdateCountersMessage> updCntrs, IgniteUuid futId) {
+        this.updCntrs = updCntrs;
+        this.futId = futId;
+    }
+
+    /**
+     * @return Partition update counters for remote node.
+     */
+    public Collection<PartitionUpdateCountersMessage> updateCounters() {
+        return updCntrs;
+    }
+
+    /**
+     * @return Sending future id.
+     */
+    public IgniteUuid futId() {
+        return futId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeIgniteUuid("futId", futId))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeCollection("updCntrs", updCntrs, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                futId = reader.readIgniteUuid("futId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                updCntrs = reader.readCollection("updCntrs", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(PartitionCountersNeighborcastRequest.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short directType() {
+        return 165;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 5;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addDeploymentInfo() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
new file mode 100644
index 0000000..547539d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/PartitionCountersNeighborcastResponse.java
@@ -0,0 +1,114 @@
+/*
+ * 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.cache.mvcc.msg;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.processors.cache.GridCacheIdMessage;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/** */
+public class PartitionCountersNeighborcastResponse extends GridCacheIdMessage {
+    /** */
+    private static final long serialVersionUID = -8731050539139260521L;
+
+    /** */
+    private IgniteUuid futId;
+
+    /** */
+    public PartitionCountersNeighborcastResponse() {
+    }
+
+    /** */
+    public PartitionCountersNeighborcastResponse(IgniteUuid futId) {
+        this.futId = futId;
+    }
+
+    /**
+     * @return Sending future id.
+     */
+    public IgniteUuid futId() {
+        return futId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeIgniteUuid("futId", futId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                futId = reader.readIgniteUuid("futId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(PartitionCountersNeighborcastResponse.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short directType() {
+        return 166;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addDeploymentInfo() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index cb682f6..240fbbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -1681,6 +1681,19 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         }
 
         /** {@inheritDoc} */
+        @Override public void finalizeUpdateCountres() {
+            try {
+                CacheDataStore delegate0 = init0(true);
+
+                if (delegate0 != null)
+                    delegate0.finalizeUpdateCountres();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
         @Override public long nextUpdateCounter() {
             try {
                 CacheDataStore delegate0 = init0(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 0d3ba75..399359b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -91,7 +91,6 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 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.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -283,7 +282,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     private volatile IgniteInternalFuture rollbackFut;
 
     /** */
-    private volatile TxCounters txCounters = new TxCounters();
+    private volatile TxCounters txCounters;
 
     /**
      * Empty constructor required for {@link Externalizable}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
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 895a9d1..75e2087 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
@@ -34,6 +35,7 @@ import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
 import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
@@ -47,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecove
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
@@ -74,6 +77,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPr
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxRemote;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastRequest;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastResponse;
 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;
@@ -257,6 +262,20 @@ public class IgniteTxHandler {
                     processCheckPreparedTxResponse(nodeId, res);
                 }
             });
+
+        ctx.io().addCacheHandler(0, PartitionCountersNeighborcastRequest.class,
+            new CI2<UUID, PartitionCountersNeighborcastRequest>() {
+                @Override public void apply(UUID nodeId, PartitionCountersNeighborcastRequest req) {
+                    processPartitionCountersRequest(nodeId, req);
+                }
+            });
+
+        ctx.io().addCacheHandler(0, PartitionCountersNeighborcastResponse.class,
+            new CI2<UUID, PartitionCountersNeighborcastResponse>() {
+                @Override public void apply(UUID nodeId, PartitionCountersNeighborcastResponse res) {
+                    processPartitionCountersResponse(nodeId, res);
+                }
+            });
     }
 
     /**
@@ -2152,4 +2171,110 @@ public class IgniteTxHandler {
 
         fut.onResult(nodeId, res);
     }
+
+    /**
+     * @param nodeId Node id.
+     * @param req Request.
+     */
+    private void processPartitionCountersRequest(UUID nodeId, PartitionCountersNeighborcastRequest req) {
+        applyPartitionsUpdatesCounters(req.updateCounters());
+
+        try {
+            ctx.io().send(nodeId, new PartitionCountersNeighborcastResponse(req.futId()), SYSTEM_POOL);
+        }
+        catch (ClusterTopologyCheckedException ignored) {
+            if (txRecoveryMsgLog.isDebugEnabled())
+                txRecoveryMsgLog.debug("Failed to send partition counters response, node left [node=" + nodeId + ']');
+        }
+        catch (IgniteCheckedException e) {
+            U.error(txRecoveryMsgLog, "Failed to send partition counters response [node=" + nodeId + ']', e);
+        }
+    }
+
+    /**
+     * @param nodeId Node id.
+     * @param res Response.
+     */
+    private void processPartitionCountersResponse(UUID nodeId, PartitionCountersNeighborcastResponse res) {
+        PartitionCountersNeighborcastFuture fut = ((PartitionCountersNeighborcastFuture)ctx.mvcc().future(res.futId()));
+
+        if (fut == null) {
+            log.warning("Failed to find future for partition counters response [futId=" + res.futId() +
+                ", node=" + nodeId + ']');
+
+            return;
+        }
+
+        fut.onResult(nodeId);
+    }
+
+    /**
+     * Applies partition counter updates for mvcc transactions.
+     *
+     * @param counters Counter values to be updated.
+     */
+    public void applyPartitionsUpdatesCounters(Iterable<PartitionUpdateCountersMessage> counters) {
+        if (counters == null)
+            return;
+
+        int cacheId = CU.UNDEFINED_CACHE_ID;
+        GridDhtPartitionTopology top = null;
+
+        for (PartitionUpdateCountersMessage counter : counters) {
+            if (counter.cacheId() != cacheId) {
+                GridCacheContext ctx0 = ctx.cacheContext(cacheId = counter.cacheId());
+
+                assert ctx0.mvccEnabled();
+
+                top = ctx0.topology();
+            }
+
+            assert top != null;
+
+            for (int i = 0; i < counter.size(); i++) {
+                GridDhtLocalPartition part = top.localPartition(counter.partition(i));
+
+                assert part != null;
+
+                part.updateCounter(counter.initialCounter(i), counter.updatesCount(i));
+            }
+        }
+    }
+
+    /**
+     * @param tx Transaction.
+     * @param node Backup node.
+     * @return Partition counters for the given backup node.
+     */
+    @Nullable public List<PartitionUpdateCountersMessage> filterUpdateCountersForBackupNode(
+        IgniteInternalTx tx, ClusterNode node) {
+        TxCounters txCntrs = tx.txCounters(false);
+
+        if (txCntrs == null || F.isEmpty(txCntrs.updateCounters()))
+            return null;
+
+        Collection<PartitionUpdateCountersMessage> updCntrs = txCntrs.updateCounters();
+
+        List<PartitionUpdateCountersMessage> res = new ArrayList<>(updCntrs.size());
+
+        AffinityTopologyVersion top = tx.topologyVersionSnapshot();
+
+        for (PartitionUpdateCountersMessage partCntrs : updCntrs) {
+            GridCacheAffinityManager affinity = ctx.cacheContext(partCntrs.cacheId()).affinity();
+
+            PartitionUpdateCountersMessage resCntrs = new PartitionUpdateCountersMessage(partCntrs.cacheId(), partCntrs.size());
+
+            for (int i = 0; i < partCntrs.size(); i++) {
+                int part = partCntrs.partition(i);
+
+                if (affinity.backupByPartition(node, part, top))
+                    resCntrs.add(part, partCntrs.initialCounter(i), partCntrs.updatesCount(i));
+            }
+
+            if (resCntrs.size() > 0)
+                res.add(resCntrs);
+        }
+
+        return res;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5939a947/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
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 27b1522..0c2ca34 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
@@ -36,12 +36,12 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
-import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
 import org.apache.ignite.internal.pagemem.wal.record.TxRecord;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObjectsReleaseFuture;
@@ -59,19 +59,18 @@ import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVe
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxFinishSync;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote;
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator;
+import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccRecoveryFinishedMessage;
 import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState;
 import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetection.TxDeadlockFuture;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -106,6 +105,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.events.EventType.EVT_TX_STARTED;
+import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE_COORDINATOR;
 import static org.apache.ignite.internal.GridTopic.TOPIC_TX;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
@@ -254,18 +254,16 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             }
         };
 
-        cctx.gridEvents().addLocalEventListener(
-            new GridLocalEventListener() {
-                @Override public void onEvent(Event evt) {
-                    assert evt instanceof DiscoveryEvent;
+        cctx.gridEvents().addDiscoveryEventListener(
+            new DiscoveryEventListener() {
+                @Override public void onEvent(DiscoveryEvent evt, DiscoCache discoCache) {
                     assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
 
-                    DiscoveryEvent discoEvt = (DiscoveryEvent)evt;
-
-                    UUID nodeId = discoEvt.eventNode().id();
+                    UUID nodeId = evt.eventNode().id();
 
                     // Wait some time in case there are some unprocessed messages from failed node.
-                    cctx.time().addTimeoutObject(new NodeFailureTimeoutObject(nodeId));
+                    cctx.time().addTimeoutObject(
+                        new NodeFailureTimeoutObject(evt.eventNode(), discoCache.mvccCoordinator()));
 
                     if (txFinishSync != null)
                         txFinishSync.onNodeLeft(nodeId);
@@ -2026,7 +2024,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      */
     public void finishTxOnRecovery(final IgniteInternalTx tx, boolean commit) {
         if (log.isInfoEnabled())
-            log.info("Finishing prepared transaction [tx=" + tx + ", commit=" + commit + ']');
+            log.info("Finishing prepared transaction [commit=" + commit + ", tx=" + tx + ']');
 
         if (!tx.markFinalizing(RECOVERY_FINISH)) {
             if (log.isInfoEnabled())
@@ -2046,10 +2044,28 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
         if (commit)
             tx.commitAsync().listen(new CommitListener(tx));
+        else if (tx.mvccSnapshot() != null && !tx.local())
+            // remote (backup) mvcc transaction sends partition counters to other backup transaction
+            // in order to keep counters consistent
+            neighborcastPartitionCountersAndRollback(tx);
         else
             tx.rollbackAsync();
     }
 
+    /** */
+    private void neighborcastPartitionCountersAndRollback(IgniteInternalTx tx) {
+        TxCounters txCounters = tx.txCounters(false);
+
+        if (txCounters == null || txCounters.updateCounters() == null)
+            tx.rollbackAsync();
+
+        PartitionCountersNeighborcastFuture fut = new PartitionCountersNeighborcastFuture(tx, cctx);
+
+        fut.listen(fut0 -> tx.rollbackAsync());
+
+        fut.init();
+    }
+
     /**
      * Commits transaction in case when node started transaction failed, but all related
      * transactions were prepared (invalidates transaction if it is not fully prepared).
@@ -2427,16 +2443,20 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * Timeout object for node failure handler.
      */
     private final class NodeFailureTimeoutObject extends GridTimeoutObjectAdapter {
-        /** Left or failed node. */
-        private final UUID evtNodeId;
+        /** */
+        private final ClusterNode node;
+        /** */
+        private final MvccCoordinator mvccCrd;
 
         /**
-         * @param evtNodeId Event node ID.
+         * @param node Failed node.
+         * @param mvccCrd Mvcc coordinator at time of node failure.
          */
-        private NodeFailureTimeoutObject(UUID evtNodeId) {
+        private NodeFailureTimeoutObject(ClusterNode node, MvccCoordinator mvccCrd) {
             super(IgniteUuid.fromUuid(cctx.localNodeId()), TX_SALVAGE_TIMEOUT);
 
-            this.evtNodeId = evtNodeId;
+            this.node = node;
+            this.mvccCrd = mvccCrd;
         }
 
         /**
@@ -2453,11 +2473,17 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                 return;
             }
 
+            UUID evtNodeId = node.id();
+
             try {
                 if (log.isDebugEnabled())
                     log.debug("Processing node failed event [locNodeId=" + cctx.localNodeId() +
                         ", failedNodeId=" + evtNodeId + ']');
 
+                // Null means that recovery voting is not needed.
+                GridCompoundFuture<IgniteInternalTx, Void> allTxFinFut = node.isClient() && mvccCrd != null
+                    ? new GridCompoundFuture<>() : null;
+
                 for (final IgniteInternalTx tx : activeTransactions()) {
                     if ((tx.near() && !tx.local()) || (tx.storeWriteThrough() && tx.masterNodeIds().contains(evtNodeId))) {
                         // Invalidate transactions.
@@ -2472,24 +2498,57 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                                 IgniteInternalFuture<?> prepFut = tx.currentPrepareFuture();
 
                                 if (prepFut != null) {
-                                    prepFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                                        @Override public void apply(IgniteInternalFuture<?> fut) {
-                                            if (tx.state() == PREPARED)
-                                                commitIfPrepared(tx, Collections.singleton(evtNodeId));
-                                            else if (tx.setRollbackOnly())
-                                                tx.rollbackAsync();
-                                        }
+                                    prepFut.listen(fut -> {
+                                        if (tx.state() == PREPARED)
+                                            commitIfPrepared(tx, Collections.singleton(evtNodeId));
+                                            // If we could not mark tx as rollback, it means that transaction is being committed.
+                                        else if (tx.setRollbackOnly())
+                                            tx.rollbackAsync();
                                     });
                                 }
-                                else {
-                                    // If we could not mark tx as rollback, it means that transaction is being committed.
-                                    if (tx.setRollbackOnly())
-                                        tx.rollbackAsync();
-                                }
+                                // If we could not mark tx as rollback, it means that transaction is being committed.
+                                else if (tx.setRollbackOnly())
+                                    tx.rollbackAsync();
                             }
                         }
+
+                        // Await only mvcc transactions initiated by failed client node.
+                        if (allTxFinFut != null && tx.eventNodeId().equals(evtNodeId)
+                            && tx.mvccSnapshot() != null)
+                            allTxFinFut.add(tx.finishFuture());
                     }
                 }
+
+                if (allTxFinFut == null)
+                    return;
+
+                allTxFinFut.markInitialized();
+
+                // Send vote to mvcc coordinator when all recovering transactions have finished.
+                allTxFinFut.listen(fut -> {
+                    // If mvcc coordinator issued snapshot for recovering transaction has failed during recovery,
+                    // then there is no need to send messages to new coordinator.
+                    try {
+                        cctx.kernalContext().io().sendToGridTopic(
+                            mvccCrd.nodeId(),
+                            TOPIC_CACHE_COORDINATOR,
+                            new MvccRecoveryFinishedMessage(evtNodeId),
+                            SYSTEM_POOL);
+                    }
+                    catch (ClusterTopologyCheckedException e) {
+                        if (log.isInfoEnabled())
+                            log.info("Mvcc coordinator issued snapshots for recovering transactions " +
+                                "has left the cluster (will ignore) [locNodeId=" + cctx.localNodeId() +
+                                    ", failedNodeId=" + evtNodeId +
+                                    ", mvccCrdNodeId=" + mvccCrd.nodeId() + ']');
+                    }
+                    catch (IgniteCheckedException e) {
+                        log.warning("Failed to notify mvcc coordinator that all recovering transactions were " +
+                            "finished [locNodeId=" + cctx.localNodeId() +
+                            ", failedNodeId=" + evtNodeId +
+                            ", mvccCrdNodeId=" + mvccCrd.nodeId() + ']', e);
+                    }
+                });
             }
             finally {
                 cctx.kernalContext().gateway().readUnlock();


[10/50] [abbrv] ignite git commit: IGNITE-9908: Python client: fixed long data type processing. This closes #5017.

Posted by sb...@apache.org.
IGNITE-9908: Python client: fixed long data type processing. This closes #5017.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1b698100
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1b698100
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1b698100

Branch: refs/heads/ignite-gg-14206
Commit: 1b6981000b6fd581608d3db7fd280d6ddd1361a2
Parents: b3688dd
Author: Dmitry Melnichuk <dm...@nobitlost.com>
Authored: Thu Oct 18 11:09:37 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 18 11:09:37 2018 +0300

----------------------------------------------------------------------
 modules/platforms/python/pyignite/datatypes/primitive.py       | 2 +-
 .../platforms/python/pyignite/datatypes/primitive_objects.py   | 2 +-
 modules/platforms/python/pyignite/datatypes/standard.py        | 6 +++---
 modules/platforms/python/pyignite/queries/__init__.py          | 6 +++---
 modules/platforms/python/setup.py                              | 6 +++++-
 5 files changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1b698100/modules/platforms/python/pyignite/datatypes/primitive.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive.py b/modules/platforms/python/pyignite/datatypes/primitive.py
index 73f096d..94c8fe3 100644
--- a/modules/platforms/python/pyignite/datatypes/primitive.py
+++ b/modules/platforms/python/pyignite/datatypes/primitive.py
@@ -66,7 +66,7 @@ class Int(Primitive):
 
 
 class Long(Primitive):
-    c_type = ctypes.c_long
+    c_type = ctypes.c_longlong
 
 
 class Float(Primitive):

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b698100/modules/platforms/python/pyignite/datatypes/primitive_objects.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive_objects.py b/modules/platforms/python/pyignite/datatypes/primitive_objects.py
index 53b54b3..4e37ce1 100644
--- a/modules/platforms/python/pyignite/datatypes/primitive_objects.py
+++ b/modules/platforms/python/pyignite/datatypes/primitive_objects.py
@@ -97,7 +97,7 @@ class IntObject(DataObject):
 
 
 class LongObject(DataObject):
-    c_type = ctypes.c_long
+    c_type = ctypes.c_longlong
     type_code = TC_LONG
     pythonic = int
     default = 0

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b698100/modules/platforms/python/pyignite/datatypes/standard.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/standard.py b/modules/platforms/python/pyignite/datatypes/standard.py
index 160e06e..6109c7a 100644
--- a/modules/platforms/python/pyignite/datatypes/standard.py
+++ b/modules/platforms/python/pyignite/datatypes/standard.py
@@ -297,7 +297,7 @@ class TimestampObject(StandardObject):
                     '_pack_': 1,
                     '_fields_': [
                         ('type_code', ctypes.c_byte),
-                        ('epoch', ctypes.c_long),
+                        ('epoch', ctypes.c_longlong),
                         ('fraction', ctypes.c_int),
                     ],
                 }
@@ -353,7 +353,7 @@ class DateObject(StandardObject):
                     '_pack_': 1,
                     '_fields_': [
                         ('type_code', ctypes.c_byte),
-                        ('epoch', ctypes.c_long),
+                        ('epoch', ctypes.c_longlong),
                     ],
                 }
             )
@@ -405,7 +405,7 @@ class TimeObject(StandardObject):
                     '_pack_': 1,
                     '_fields_': [
                         ('type_code', ctypes.c_byte),
-                        ('value', ctypes.c_long),
+                        ('value', ctypes.c_longlong),
                     ],
                 }
             )

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b698100/modules/platforms/python/pyignite/queries/__init__.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/queries/__init__.py b/modules/platforms/python/pyignite/queries/__init__.py
index f43d60e..2c2d254 100644
--- a/modules/platforms/python/pyignite/queries/__init__.py
+++ b/modules/platforms/python/pyignite/queries/__init__.py
@@ -54,7 +54,7 @@ class Response:
                     '_pack_': 1,
                     '_fields_': [
                         ('length', ctypes.c_int),
-                        ('query_id', ctypes.c_long),
+                        ('query_id', ctypes.c_longlong),
                         ('status_code', ctypes.c_int),
                     ],
                 },
@@ -239,7 +239,7 @@ class Query:
                     '_fields_': [
                         ('length', ctypes.c_int),
                         ('op_code', ctypes.c_short),
-                        ('query_id', ctypes.c_long),
+                        ('query_id', ctypes.c_longlong),
                     ],
                 },
             )
@@ -309,7 +309,7 @@ class ConfigQuery(Query):
                     '_fields_': [
                         ('length', ctypes.c_int),
                         ('op_code', ctypes.c_short),
-                        ('query_id', ctypes.c_long),
+                        ('query_id', ctypes.c_longlong),
                         ('config_length', ctypes.c_int),
                     ],
                 },

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b698100/modules/platforms/python/setup.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/setup.py b/modules/platforms/python/setup.py
index 403b170..5d14dae 100644
--- a/modules/platforms/python/setup.py
+++ b/modules/platforms/python/setup.py
@@ -55,7 +55,11 @@ requirement_sections = [
 requirements = defaultdict(list)
 
 for section in requirement_sections:
-    with open('requirements/{}.txt'.format(section), 'r') as requirements_file:
+    with open(
+        'requirements/{}.txt'.format(section),
+        'r',
+        encoding='utf-8',
+    ) as requirements_file:
         for line in requirements_file.readlines():
             line = line.strip('\n')
             if is_a_requirement(line):


[40/50] [abbrv] ignite git commit: IGNITE-8006 Removed non-representative flaky test

Posted by sb...@apache.org.
IGNITE-8006 Removed non-representative flaky test


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1f068e01
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1f068e01
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1f068e01

Branch: refs/heads/ignite-gg-14206
Commit: 1f068e01cb0479975c8c6c1a56d46671bee0e515
Parents: 75e414a
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Oct 23 11:01:40 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Oct 23 11:02:51 2018 +0300

----------------------------------------------------------------------
 .../distributed/CacheStartInParallelTest.java   | 219 -------------------
 .../testsuites/IgniteCacheTestSuite7.java       |   3 -
 2 files changed, 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f068e01/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java
deleted file mode 100644
index 4e30d1c..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/*
- * 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.cache.distributed;
-
-import java.util.ArrayList;
-import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.DataRegionConfiguration;
-import org.apache.ignite.configuration.DataStorageConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.WALMode;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class CacheStartInParallelTest extends GridCommonAbstractTest {
-    /** */
-    private static final int CACHES_COUNT = 40;
-
-    /** */
-    private static final String STATIC_CACHE_PREFIX = "static-cache-";
-
-    /** */
-    private static final String DYNAMIC_CACHE_PREFIX = "dynamic-cache-";
-
-    /** */
-    private static boolean isStaticCache = true;
-
-    /** */
-    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setSystemThreadPoolSize(10);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
-
-        long sz = 100 * 1024 * 1024;
-
-        DataStorageConfiguration memCfg = new DataStorageConfiguration().setPageSize(1024)
-            .setDefaultDataRegionConfiguration(
-                new DataRegionConfiguration().setPersistenceEnabled(true).setInitialSize(sz).setMaxSize(sz))
-            .setWalMode(WALMode.LOG_ONLY).setCheckpointFrequency(24L * 60 * 60 * 1000);
-
-        cfg.setDataStorageConfiguration(memCfg);
-
-        if (isStaticCache) {
-            ArrayList<Object> staticCaches = new ArrayList<>(CACHES_COUNT);
-
-            for (int i = 0; i < CACHES_COUNT; i++)
-                staticCaches.add(cacheConfiguration(STATIC_CACHE_PREFIX + i));
-
-            cfg.setCacheConfiguration(staticCaches.toArray(new CacheConfiguration[CACHES_COUNT]));
-        }
-
-        return cfg;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     */
-    private CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration cfg = defaultCacheConfiguration();
-
-        cfg.setName(cacheName);
-        cfg.setBackups(1);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        cleanupTestData();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        cleanupTestData();
-    }
-
-    /** */
-    private void cleanupTestData() throws Exception {
-        stopAllGrids();
-
-        cleanPersistenceDir();
-
-        System.clearProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL);
-
-        isStaticCache = true;
-    }
-
-    /**
-     * Checking that start static caches in parallel faster than consistenly.
-     *
-     * @throws Exception if fail.
-     */
-    public void testParallelizationAcceleratesStartOfStaticCaches() throws Exception {
-        //start caches consistently.
-        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "false");
-
-        long startTime = System.currentTimeMillis();
-
-        IgniteEx igniteEx = startGrid(0);
-
-        igniteEx.cluster().active(true);
-
-        long totalStartTimeConsistently = System.currentTimeMillis() - startTime;
-
-        //check cache started.
-        for (int i = 0; i < CACHES_COUNT; i++)
-            igniteEx.cache(STATIC_CACHE_PREFIX + i).put(i, i);
-
-        stopAllGrids();
-
-        //start caches in parallel.
-        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "true");
-
-        startTime = System.currentTimeMillis();
-
-        igniteEx = startGrid(0);
-
-        igniteEx.cluster().active(true);
-
-        long totalStartTimeInParallel = System.currentTimeMillis() - startTime;
-
-        for (int i = 0; i < CACHES_COUNT; i++)
-            igniteEx.cache(STATIC_CACHE_PREFIX + i).put(i, i);
-
-        stopAllGrids();
-
-        assertTrue("Consistently cache stat time : " + totalStartTimeConsistently +
-                "Parallelization cache stat time : " + totalStartTimeInParallel,
-            totalStartTimeConsistently > totalStartTimeInParallel);
-    }
-
-    /**
-     * Checking that start dynamic caches in parallel faster than consistenly.
-     *
-     * @throws Exception if fail.
-     */
-    public void testParallelizationAcceleratesStartOfCaches2() throws Exception {
-        //prepare dynamic caches.
-        isStaticCache = false;
-
-        IgniteEx igniteEx = startGrid(0);
-
-        igniteEx.cluster().active(true);
-
-        for (int i = 0; i < CACHES_COUNT; i++)
-            igniteEx.getOrCreateCache(DYNAMIC_CACHE_PREFIX + i);
-
-        stopAllGrids();
-
-        //start caches consistently.
-        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "false");
-
-        igniteEx = startGrid(0);
-        long startTime = System.currentTimeMillis();
-
-        igniteEx.cluster().active(true);
-
-        long totalStartTimeConsistently = System.currentTimeMillis() - startTime;
-
-        for (int i = 0; i < CACHES_COUNT; i++)
-            igniteEx.cache(DYNAMIC_CACHE_PREFIX + i);
-
-        stopAllGrids();
-
-        //start caches in parallel.
-        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "true");
-
-        startTime = System.currentTimeMillis();
-
-        igniteEx = startGrid(0);
-
-        igniteEx.cluster().active(true);
-
-        long totalStartTimeInParallel = System.currentTimeMillis() - startTime;
-
-        for (int i = 0; i < CACHES_COUNT; i++)
-            igniteEx.cache(DYNAMIC_CACHE_PREFIX + i).put(i, i);
-
-        stopAllGrids();
-
-        assertTrue("Consistently cache stat time : " + totalStartTimeConsistently +
-                "Parallelization cache stat time : " + totalStartTimeInParallel,
-            totalStartTimeConsistently > totalStartTimeInParallel);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f068e01/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
index 0381a1f..d0734a8 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
@@ -36,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartition
 import org.apache.ignite.internal.processors.cache.distributed.CacheDataLossOnPartitionMoveTest;
 import org.apache.ignite.internal.processors.cache.distributed.CachePageWriteLockUnlockTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheRentingStateRepairTest;
-import org.apache.ignite.internal.processors.cache.distributed.CacheStartInParallelTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheStartWithLoadTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingPartitionCountersTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingWithAsyncClearingTest;
@@ -100,8 +99,6 @@ public class IgniteCacheTestSuite7 extends TestSuite {
 
         suite.addTestSuite(CacheRentingStateRepairTest.class);
 
-        suite.addTestSuite(CacheStartInParallelTest.class);
-
         suite.addTestSuite(TransactionIntegrityWithPrimaryIndexCorruptionTest.class);
         suite.addTestSuite(CacheDataLossOnPartitionMoveTest.class);
 


[48/50] [abbrv] ignite git commit: IGNITE-9632: SQL: support IN statement with constants/params for partition pruning. This closes #4857.

Posted by sb...@apache.org.
IGNITE-9632: SQL: support IN statement with constants/params for partition pruning. This closes #4857.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/324e6e38
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/324e6e38
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/324e6e38

Branch: refs/heads/ignite-gg-14206
Commit: 324e6e38eb1ef042bbf3a506d523201266893050
Parents: d669da1
Author: devozerov <vo...@gridgain.com>
Authored: Tue Oct 23 15:16:56 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 23 15:16:56 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java  | 4 ++++
 .../apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java | 3 ---
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/324e6e38/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 7633d2a..f29a79b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -186,6 +186,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
+import org.apache.ignite.internal.processors.query.h2.twostep.InOperationExtractPartitionSelfTest;
 import org.apache.ignite.internal.processors.sql.IgniteCachePartitionedAtomicColumnConstraintsTest;
 import org.apache.ignite.internal.processors.sql.IgniteCachePartitionedTransactionalColumnConstraintsTest;
 import org.apache.ignite.internal.processors.sql.IgniteCacheReplicatedAtomicColumnConstraintsTest;
@@ -486,6 +487,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteSqlGroupConcatCollocatedTest.class);
         suite.addTestSuite(IgniteSqlGroupConcatNotCollocatedTest.class);
 
+        // Partition pruning.
+        suite.addTestSuite(InOperationExtractPartitionSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/324e6e38/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index ac8d10a..dba046b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.processors.query.h2.twostep.CacheQueryMemoryLe
 import org.apache.ignite.internal.processors.query.h2.twostep.CreateTableWithDateKeySelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.DisappearedCacheCauseRetryMessageSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.DisappearedCacheWasNotFoundMessageSelfTest;
-import org.apache.ignite.internal.processors.query.h2.twostep.InOperationExtractPartitionSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.NonCollocatedRetryMessageSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.RetryCauseMessageSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.TableViewSubquerySelfTest;
@@ -124,8 +123,6 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
         suite.addTestSuite(DisappearedCacheCauseRetryMessageSelfTest.class);
         suite.addTestSuite(DisappearedCacheWasNotFoundMessageSelfTest.class);
 
-        suite.addTestSuite(InOperationExtractPartitionSelfTest.class);
-
         suite.addTestSuite(TableViewSubquerySelfTest.class);
 
         return suite;


[43/50] [abbrv] ignite git commit: IGNITE-9911: MVCC: Fixed a hang during vacuum. This closes #5035.

Posted by sb...@apache.org.
IGNITE-9911: MVCC: Fixed a hang during vacuum. This closes #5035.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b1584a8f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b1584a8f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b1584a8f

Branch: refs/heads/ignite-gg-14206
Commit: b1584a8f2636d390b3f8cc754ce1195ccc797807
Parents: 962d6a2
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Tue Oct 23 12:05:23 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 23 12:05:23 2018 +0300

----------------------------------------------------------------------
 .../cache/mvcc/MvccProcessorImpl.java           | 228 ++++++++-----------
 .../cache/mvcc/CacheMvccAbstractTest.java       |  21 +-
 ...cheMvccSelectForUpdateQueryAbstractTest.java |  11 +-
 3 files changed, 115 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b1584a8f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index 9fcafb0..e58151f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -90,8 +90,8 @@ import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.GridCursor;
-import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -115,7 +115,6 @@ import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE_COORDINATOR;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
-import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker.MVCC_TRACKER_ID_NA;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA;
@@ -181,9 +180,6 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
      */
     private final Object mux = new Object();
 
-    /** For tests only. */
-    private volatile Throwable vacuumError;
-
     /** */
     private final GridAtomicLong futIdCntr = new GridAtomicLong(0);
 
@@ -1136,8 +1132,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
             crdVer == 0 && ctx.localNodeId().equals(crd0.nodeId()))
             return new GridFinishedFuture<>(new VacuumMetrics());
 
-        final GridCompoundIdentityFuture<VacuumMetrics> res =
-            new GridCompoundIdentityFuture<>(new VacuumMetricsReducer());
+        final GridFutureAdapter<VacuumMetrics> res = new GridFutureAdapter<>();
 
         MvccSnapshot snapshot;
 
@@ -1174,28 +1169,10 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     }
 
     /**
-     * For tests only.
-     *
-     * @return Vacuum error.
-     */
-    Throwable vacuumError() {
-        return vacuumError;
-    }
-
-    /**
-     * For tests only.
-     *
-     * @param e Vacuum error.
-     */
-    void vacuumError(Throwable e) {
-        this.vacuumError = e;
-    }
-
-    /**
      * @param res Result.
      * @param snapshot Snapshot.
      */
-    private void continueRunVacuum(GridCompoundIdentityFuture<VacuumMetrics> res, MvccSnapshot snapshot) {
+    private void continueRunVacuum(GridFutureAdapter<VacuumMetrics> res, MvccSnapshot snapshot) {
         ackTxCommit(snapshot)
             .listen(new IgniteInClosure<IgniteInternalFuture>() {
                 @Override public void apply(IgniteInternalFuture fut) {
@@ -1220,23 +1197,45 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                                     return;
                                 }
 
+                                GridCompoundIdentityFuture<VacuumMetrics> res0 =
+                                    new GridCompoundIdentityFuture<VacuumMetrics>(new VacuumMetricsReducer()) {
+                                        /** {@inheritDoc} */
+                                        @Override protected void logError(IgniteLogger log, String msg, Throwable e) {
+                                            // no-op
+                                        }
+
+                                        /** {@inheritDoc} */
+                                        @Override protected void logDebug(IgniteLogger log, String msg) {
+                                            // no-op
+                                        }
+                                    };
+
                                 for (CacheGroupContext grp : ctx.cache().cacheGroups()) {
                                     if (grp.mvccEnabled()) {
-                                        for (GridDhtLocalPartition part : grp.topology().localPartitions()) {
-                                            VacuumTask task = new VacuumTask(snapshot, part);
+                                        grp.topology().readLock();
+
+                                        try {
+                                            for (GridDhtLocalPartition part : grp.topology().localPartitions()) {
+                                                VacuumTask task = new VacuumTask(snapshot, part);
 
-                                            cleanupQueue.offer(task);
+                                                cleanupQueue.offer(task);
 
-                                            res.add(task);
+                                                res0.add(task);
+                                            }
+                                        }
+                                        finally {
+                                            grp.topology().readUnlock();
                                         }
                                     }
                                 }
-                            }
 
-                            res.listen(new CI1<IgniteInternalFuture<VacuumMetrics>>() {
-                                @Override public void apply(IgniteInternalFuture<VacuumMetrics> fut) {
+                                res0.markInitialized();
+
+                                res0.listen(future -> {
+                                    VacuumMetrics metrics = null; Throwable ex = null;
+
                                     try {
-                                        VacuumMetrics metrics = fut.get();
+                                        metrics = future.get();
 
                                         if (U.assertionsEnabled()) {
                                             MvccCoordinator crd = currentCoordinator();
@@ -1260,14 +1259,17 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                                     catch (NodeStoppingException ignored) {
                                         if (log.isDebugEnabled())
                                             log.debug("Cannot complete vacuum (node is stopping).");
+
+                                        metrics = new VacuumMetrics();
                                     }
                                     catch (Throwable e) {
-                                        U.error(log, "Vacuum error.", e);
+                                        ex = new GridClosureException(e);
                                     }
-                                }
-                            });
 
-                            res.markInitialized();
+                                    res.onDone(metrics, ex);
+                                });
+                            }
+
                         }
                         catch (Throwable e) {
                             completeWithException(res, e);
@@ -2138,13 +2140,13 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                     }
                 }
                 catch (IgniteInterruptedCheckedException e) {
-                    throw e;
+                    throw e; // Cancelled.
                 }
                 catch (Throwable e) {
-                    prc.vacuumError(e);
-
                     if (e instanceof Error)
                         throw (Error) e;
+
+                    U.error(log, "Vacuum error.", e);
                 }
 
                 long delay = nextScheduledTime - U.currentTimeMillis();
@@ -2179,20 +2181,29 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                 VacuumTask task = cleanupQueue.take();
 
                 try {
-                    if (task.part().state() != OWNING) {
-                        task.part().group().preloader().rebalanceFuture()
-                            .listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
-                                @Override public void apply(IgniteInternalFuture<Boolean> future) {
-                                    cleanupQueue.add(task);
-                                }
-                            });
+                    switch (task.part().state()) {
+                        case EVICTED:
+                        case RENTING:
+                            task.onDone(new VacuumMetrics());
 
-                        continue;
-                    }
+                            break;
+                        case MOVING:
+                            task.part().group().preloader().rebalanceFuture().listen(f -> cleanupQueue.add(task));
 
-                    task.onDone(processPartition(task));
+                            break;
+                        case OWNING:
+                            task.onDone(processPartition(task));
+
+                            break;
+                        case LOST:
+                            task.onDone(new IgniteCheckedException("Partition is lost."));
+
+                            break;
+                    }
                 }
                 catch (IgniteInterruptedCheckedException e) {
+                    task.onDone(e);
+
                     throw e; // Cancelled.
                 }
                 catch (Throwable e) {
@@ -2217,7 +2228,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
             VacuumMetrics metrics = new VacuumMetrics();
 
-            if (!canRunVacuum(part, null) || !part.reserve())
+            if (!part.reserve())
                 return metrics;
 
             int curCacheId = CU.UNDEFINED_CACHE_ID;
@@ -2289,63 +2300,11 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
                 return metrics;
             }
-            catch (Exception e) {
-                if (canRunVacuum(part, curCacheId))
-                    throw e; // Unexpected error.
-
-                U.warn(log, "Error occurred during the vacuum. Skip vacuuming for the current partition. " +
-                    "[cacheId=" + curCacheId + ", part=" + part + ", err=" + e.getMessage() + ']', e);
-
-                return new VacuumMetrics();
-            }
             finally {
                 part.release();
             }
         }
 
-        /**
-         * @param part Partition.
-         * @param cacheId Cache id.
-         * @return {@code True} if we can vacuum given partition.
-         */
-        private boolean canRunVacuum(GridDhtLocalPartition part, Integer cacheId) {
-            if (part == null || part.state() != OWNING)
-                return false;
-
-            CacheGroupContext grp = part.group();
-
-            assert grp != null;
-
-            List<GridCacheContext> caches = grp.caches();
-
-            if (F.isEmpty(caches))
-                return false;
-
-            if (grp.shared().kernalContext().isStopping())
-                return false;
-
-            if (cacheId == null && grp.sharedGroup())
-                return true; // Cache context is unknown, but we can try to run vacuum.
-
-            GridCacheContext ctx0;
-
-            if (grp.sharedGroup()) {
-                assert cacheId != null && cacheId != CU.UNDEFINED_CACHE_ID;
-
-                if (!grp.cacheIds().contains(cacheId))
-                    return false;
-
-                ctx0 = grp.shared().cacheContext(cacheId);
-            }
-            else
-                ctx0 = caches.get(0);
-
-            if (ctx0 == null)
-                return false;
-
-            return !grp.shared().closed(ctx0);
-        }
-
         /** */
         @SuppressWarnings("unchecked")
         @NotNull private Object addRest(@Nullable Object rest, MvccDataRow row) {
@@ -2417,47 +2376,54 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
             Object rest, GridCacheContext cctx, VacuumMetrics metrics) throws IgniteCheckedException {
             assert key != null && cctx != null && (!F.isEmpty(cleanupRows) || rest != null);
 
-            long cleanupStartNanoTime = System.nanoTime();
+            cctx.gate().enter();
 
-            GridCacheEntryEx entry = cctx.cache().entryEx(key);
+            try {
+                long cleanupStartNanoTime = System.nanoTime();
 
-            while (true) {
-                entry.lockEntry();
+                GridCacheEntryEx entry = cctx.cache().entryEx(key);
 
-                if (!entry.obsolete())
-                    break;
+                while (true) {
+                    entry.lockEntry();
 
-                entry.unlockEntry();
+                    if (!entry.obsolete())
+                        break;
 
-                entry = cctx.cache().entryEx(key);
-            }
+                    entry.unlockEntry();
 
-            cctx.shared().database().checkpointReadLock();
+                    entry = cctx.cache().entryEx(key);
+                }
 
-            int cleaned = 0;
+                cctx.shared().database().checkpointReadLock();
 
-            try {
-                if (cleanupRows != null)
-                    cleaned = part.dataStore().cleanup(cctx, cleanupRows);
+                int cleaned = 0;
+
+                try {
+                    if (cleanupRows != null)
+                        cleaned = part.dataStore().cleanup(cctx, cleanupRows);
 
-                if (rest != null) {
-                    if (rest.getClass() == ArrayList.class) {
-                        for (MvccDataRow row : ((List<MvccDataRow>)rest)) {
-                            part.dataStore().updateTxState(cctx, row);
+                    if (rest != null) {
+                        if (rest.getClass() == ArrayList.class) {
+                            for (MvccDataRow row : ((List<MvccDataRow>)rest)) {
+                                part.dataStore().updateTxState(cctx, row);
+                            }
                         }
+                        else
+                            part.dataStore().updateTxState(cctx, (MvccDataRow)rest);
                     }
-                    else
-                        part.dataStore().updateTxState(cctx, (MvccDataRow)rest);
                 }
-            }
-            finally {
-                cctx.shared().database().checkpointReadUnlock();
+                finally {
+                    cctx.shared().database().checkpointReadUnlock();
 
-                entry.unlockEntry();
-                cctx.evicts().touch(entry, AffinityTopologyVersion.NONE);
+                    entry.unlockEntry();
+                    cctx.evicts().touch(entry, AffinityTopologyVersion.NONE);
 
-                metrics.addCleanupNanoTime(System.nanoTime() - cleanupStartNanoTime);
-                metrics.addCleanupRowsCnt(cleaned);
+                    metrics.addCleanupNanoTime(System.nanoTime() - cleanupStartNanoTime);
+                    metrics.addCleanupRowsCnt(cleaned);
+                }
+            }
+            finally {
+                cctx.gate().leave();
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1584a8f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
index ec6b78a..57f714a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
@@ -1532,14 +1532,21 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     protected void verifyOldVersionsCleaned() throws Exception {
-        runVacuumSync();
+        boolean retry;
 
-        awaitPartitionMapExchange();
+        try {
+            runVacuumSync();
 
-        // Check versions.
-        boolean cleaned = checkOldVersions(false);
+            // Check versions.
+            retry = !checkOldVersions(false);
+        }
+        catch (Exception e) {
+            U.warn(log(), "Failed to perform vacuum, will retry.", e);
 
-        if (!cleaned) { // Retry on a stable topology with a newer snapshot.
+            retry = true;
+        }
+
+        if (retry) { // Retry on a stable topology with a newer snapshot.
             awaitPartitionMapExchange();
 
             runVacuumSync();
@@ -1605,10 +1612,6 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
 
                 assert GridTestUtils.getFieldValue(crd, "txLog") != null;
 
-                Throwable vacuumError = crd.vacuumError();
-
-                assertNull(X.getFullStackTrace(vacuumError), vacuumError);
-
                 fut.add(crd.runVacuum());
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1584a8f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java
index c6584e9..16b45ab 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.util.typedef.X;
@@ -57,7 +58,7 @@ public abstract class CacheMvccSelectForUpdateQueryAbstractTest extends CacheMvc
 
         disableScheduledVacuum = getName().equals("testSelectForUpdateAfterAbortedTx");
 
-        startGrids(3);
+        IgniteEx grid = startGrid(0);
 
         CacheConfiguration seg = new CacheConfiguration("segmented*");
 
@@ -66,11 +67,9 @@ public abstract class CacheMvccSelectForUpdateQueryAbstractTest extends CacheMvc
         if (seg.getCacheMode() == PARTITIONED)
             seg.setQueryParallelism(4);
 
-        grid(0).addCacheConfiguration(seg);
+        grid.addCacheConfiguration(seg);
 
-        Thread.sleep(1000L);
-
-        try (Connection c = connect(grid(0))) {
+        try (Connection c = connect(grid)) {
             execute(c, "create table person (id int primary key, firstName varchar, lastName varchar) " +
                 "with \"atomicity=transactional_snapshot,cache_name=Person\"");
 
@@ -90,6 +89,8 @@ public abstract class CacheMvccSelectForUpdateQueryAbstractTest extends CacheMvc
                 tx.commit();
             }
         }
+
+        startGridsMultiThreaded(1, 2);
     }
 
     /**


[07/50] [abbrv] ignite git commit: IGNITE-9756 Fixed partition eviction deduplication logic - Fixes #4936.

Posted by sb...@apache.org.
IGNITE-9756 Fixed partition eviction deduplication logic - Fixes #4936.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/aec3f91c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/aec3f91c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/aec3f91c

Branch: refs/heads/ignite-gg-14206
Commit: aec3f91c22c26d3ceb56769d6d7a13a92c9036d4
Parents: 03cac75
Author: pereslegin-pa <xx...@gmail.com>
Authored: Wed Oct 17 19:58:29 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Oct 17 19:58:29 2018 +0300

----------------------------------------------------------------------
 .../dht/topology/PartitionsEvictManager.java    | 49 ++++++++------------
 1 file changed, 20 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/aec3f91c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
index 7f2a2a7..404e194 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.topology;
 
 import java.util.Collection;
 import java.util.Comparator;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
@@ -31,7 +32,6 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -111,28 +111,28 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
      * @param part Partition to evict.
      */
     public void evictPartitionAsync(CacheGroupContext grp, GridDhtLocalPartition part) {
-        // Check node stop.
-        if (sharedEvictionContext.shouldStop())
-            return;
-
         GroupEvictionContext groupEvictionContext = evictionGroupsMap.computeIfAbsent(
             grp.groupId(), (k) -> new GroupEvictionContext(grp));
 
-        PartitionEvictionTask evictionTask = groupEvictionContext.createEvictPartitionTask(part);
-
-        if (evictionTask == null)
+        // Check node stop.
+        if (groupEvictionContext.shouldStop())
             return;
 
-        if (log.isDebugEnabled())
-            log.debug("Partition has been scheduled for eviction [grp=" + grp.cacheOrGroupName()
-                + ", p=" + part.id() + ", state=" + part.state() + "]");
-
         int bucket;
 
         synchronized (mux) {
-            bucket = evictionQueue.offer(evictionTask);
+            if (!groupEvictionContext.partIds.add(part.id()))
+                return;
+
+            bucket = evictionQueue.offer(new PartitionEvictionTask(part, groupEvictionContext));
         }
 
+        groupEvictionContext.totalTasks.incrementAndGet();
+
+        if (log.isDebugEnabled())
+            log.debug("Partition has been scheduled for eviction [grp=" + grp.cacheOrGroupName()
+                + ", p=" + part.id() + ", state=" + part.state() + "]");
+
         scheduleNextPartitionEviction(bucket);
     }
 
@@ -271,7 +271,7 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
         private final CacheGroupContext grp;
 
         /** Deduplicate set partition ids. */
-        private final Set<Integer> partIds = new GridConcurrentHashSet<>();
+        private final Set<Integer> partIds = new HashSet<>();
 
         /** Future for currently running partition eviction task. */
         private final Map<Integer, IgniteInternalFuture<?>> partsEvictFutures = new ConcurrentHashMap<>();
@@ -299,19 +299,6 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
 
         /**
          *
-         * @param part Grid local partition.
-         */
-        private PartitionEvictionTask createEvictPartitionTask(GridDhtLocalPartition part){
-            if (shouldStop() || !partIds.add(part.id()))
-                return null;
-
-            totalTasks.incrementAndGet();
-
-            return new PartitionEvictionTask(part, this);
-        }
-
-        /**
-         *
          * @param task Partition eviction task.
          */
         private synchronized void taskScheduled(PartitionEvictionTask task) {
@@ -324,6 +311,8 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
 
             int partId = task.part.id();
 
+            partIds.remove(partId);
+
             partsEvictFutures.put(partId, fut);
 
             fut.listen(f -> {
@@ -429,12 +418,14 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
                     if (part.state() == GridDhtPartitionState.EVICTED && part.markForDestroy())
                         part.destroy();
                 }
-                else // Re-offer partition if clear was unsuccessful due to partition reservation.
-                    evictionQueue.offer(this);
 
                 // Complete eviction future before schedule new to prevent deadlock with
                 // simultaneous eviction stopping and scheduling new eviction.
                 finishFut.onDone();
+
+                // Re-offer partition if clear was unsuccessful due to partition reservation.
+                if (!success)
+                    evictPartitionAsync(groupEvictionCtx.grp, part);
             }
             catch (Throwable ex) {
                 finishFut.onDone(ex);


[33/50] [abbrv] ignite git commit: IGNITE-8006 Parallelize cache groups start - Fixes #4752.

Posted by sb...@apache.org.
IGNITE-8006 Parallelize cache groups start - Fixes #4752.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e1f8f46f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e1f8f46f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e1f8f46f

Branch: refs/heads/ignite-gg-14206
Commit: e1f8f46f90868d377bc764b74d07812150218c71
Parents: e5a4672
Author: Anton Kalashnikov <ka...@yandex.ru>
Authored: Mon Oct 22 16:27:49 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 16:27:49 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../cache/CacheAffinitySharedManager.java       | 115 ++--
 .../processors/cache/GridCacheIoManager.java    |  22 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    | 523 ++++++++++++++-----
 .../processors/cache/StartCacheInfo.java        | 113 ++++
 .../GridDhtPartitionsExchangeFuture.java        |  78 ++-
 .../ignite/internal/util/IgniteUtils.java       | 104 ++--
 .../internal/util/InitializationProtector.java  |  79 +++
 .../util/lang/IgniteThrowableConsumer.java      |  37 ++
 .../util/lang/IgniteThrowableRunner.java        |  30 ++
 .../distributed/CacheStartInParallelTest.java   | 219 ++++++++
 .../IgniteCrossCacheTxStoreSelfTest.java        |  44 +-
 .../internal/util/IgniteUtilsSelfTest.java      |  74 +++
 .../testsuites/IgniteCacheTestSuite7.java       |   3 +
 15 files changed, 1165 insertions(+), 287 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 521222c..6afe244 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1030,6 +1030,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_REBALANCE_THROTTLE_OVERRIDE = "IGNITE_REBALANCE_THROTTLE_OVERRIDE";
 
     /**
+     * Enables start caches in parallel.
+     *
+     * Default is {@code true}.
+     */
+    public static final String IGNITE_ALLOW_START_CACHES_IN_PARALLEL = "IGNITE_ALLOW_START_CACHES_IN_PARALLEL";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
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 cedbde1..6e10c00 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
@@ -17,12 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import javax.cache.CacheException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -30,7 +32,6 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
@@ -427,25 +428,43 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         Map<Integer, GridDhtAssignmentFetchFuture> fetchFuts = U.newHashMap(startDescs.size());
 
-        Set<String> startedCaches = U.newHashSet(startDescs.size());
-
         Map<Integer, Boolean> startedInfos = U.newHashMap(startDescs.size());
 
-        for (DynamicCacheDescriptor desc : startDescs) {
-            try {
-                startedCaches.add(desc.cacheName());
+        List<StartCacheInfo> startCacheInfos = startDescs.stream()
+            .map(desc -> {
+                DynamicCacheChangeRequest changeReq = startReqs.get(desc.cacheName());
 
-                DynamicCacheChangeRequest startReq = startReqs.get(desc.cacheName());
-
-                cctx.cache().prepareCacheStart(
-                    desc.cacheConfiguration(),
+                return new StartCacheInfo(
                     desc,
-                    startReq.nearCacheConfiguration(),
+                    changeReq.nearCacheConfiguration(),
                     topVer,
-                    startReq.disabledAfterStart()
+                    changeReq.disabledAfterStart()
                 );
+            })
+            .collect(Collectors.toList());
+
+        Set<String> startedCaches = startCacheInfos.stream()
+            .map(info -> info.getCacheDescriptor().cacheName())
+            .collect(Collectors.toSet());
+
+        try {
+            cctx.cache().prepareStartCaches(startCacheInfos);
+        }
+        catch (IgniteCheckedException e) {
+            cctx.cache().closeCaches(startedCaches, false);
+
+            cctx.cache().completeClientCacheChangeFuture(msg.requestId(), e);
+
+            return null;
+        }
+
+        for (StartCacheInfo startCacheInfo : startCacheInfos) {
+            try {
+                DynamicCacheDescriptor desc = startCacheInfo.getCacheDescriptor();
+
+                startedCaches.add(desc.cacheName());
 
-                startedInfos.put(desc.cacheId(), startReq.nearCacheConfiguration() != null);
+                startedInfos.put(desc.cacheId(), startCacheInfo.getReqNearCfg() != null);
 
                 CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
 
@@ -860,6 +879,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         long time = System.currentTimeMillis();
 
+        Map<StartCacheInfo, DynamicCacheChangeRequest> startCacheInfos = new LinkedHashMap<>();
+
         for (ExchangeActions.CacheActionData action : exchActions.cacheStartRequests()) {
             DynamicCacheDescriptor cacheDesc = action.descriptor();
 
@@ -895,29 +916,41 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 }
             }
 
-            try {
-                if (startCache) {
-                    cctx.cache().prepareCacheStart(
+            if (startCache) {
+                startCacheInfos.put(
+                    new StartCacheInfo(
                         req.startCacheConfiguration(),
                         cacheDesc,
                         nearCfg,
                         evts.topologyVersion(),
                         req.disabledAfterStart()
-                    );
-
-                    if (fut.cacheAddedOnExchange(cacheDesc.cacheId(), cacheDesc.receivedFrom())) {
-                        if (fut.events().discoveryCache().cacheGroupAffinityNodes(cacheDesc.groupId()).isEmpty())
-                            U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName());
-                    }
-                }
+                    ),
+                    req
+                );
             }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to initialize cache. Will try to rollback cache start routine. " +
-                    "[cacheName=" + req.cacheName() + ']', e);
+        }
+
+        Map<StartCacheInfo, IgniteCheckedException> failedCaches = cctx.cache().prepareStartCachesIfPossible(startCacheInfos.keySet());
 
-                cctx.cache().closeCaches(Collections.singleton(req.cacheName()), false);
+        failedCaches.forEach((cacheInfo, exception) -> {
+            U.error(log, "Failed to initialize cache. Will try to rollback cache start routine. " +
+                "[cacheName=" + cacheInfo.getStartedConfiguration().getName() + ']', exception);
 
-                cctx.cache().completeCacheStartFuture(req, false, e);
+            cctx.cache().closeCaches(Collections.singleton(cacheInfo.getStartedConfiguration().getName()), false);
+
+            cctx.cache().completeCacheStartFuture(startCacheInfos.get(cacheInfo), false, exception);
+        });
+
+        Set<StartCacheInfo> failedCacheInfos = failedCaches.keySet();
+
+        List<StartCacheInfo> cacheInfos = startCacheInfos.keySet().stream()
+            .filter(failedCacheInfos::contains)
+            .collect(Collectors.toList());
+
+        for (StartCacheInfo info : cacheInfos) {
+            if (fut.cacheAddedOnExchange(info.getCacheDescriptor().cacheId(), info.getCacheDescriptor().receivedFrom())) {
+                if (fut.events().discoveryCache().cacheGroupAffinityNodes(info.getCacheDescriptor().groupId()).isEmpty())
+                    U.quietAndWarn(log, "No server nodes found for cache client: " + info.getCacheDescriptor().cacheName());
             }
         }
 
@@ -952,22 +985,20 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         U.doInParallel(
             cctx.kernalContext().getSystemExecutorService(),
             startedGroups,
-            new IgniteInClosureX<CacheGroupDescriptor>() {
-                @Override public void applyx(CacheGroupDescriptor grpDesc) throws IgniteCheckedException {
-                    if (crd)
-                        initStartedGroupOnCoordinator(fut, grpDesc);
-                    else {
-                        CacheGroupContext grp = cctx.cache().cacheGroup(grpDesc.groupId());
+            grpDesc -> {
+                if (crd)
+                    initStartedGroupOnCoordinator(fut, grpDesc);
+                else {
+                    CacheGroupContext grp = cctx.cache().cacheGroup(grpDesc.groupId());
 
-                        if (grp != null && !grp.isLocal() && grp.localStartVersion().equals(fut.initialVersion())) {
-                            assert grp.affinity().lastVersion().equals(AffinityTopologyVersion.NONE) : grp.affinity().lastVersion();
+                    if (grp != null && !grp.isLocal() && grp.localStartVersion().equals(fut.initialVersion())) {
+                        assert grp.affinity().lastVersion().equals(AffinityTopologyVersion.NONE) : grp.affinity().lastVersion();
 
-                            initAffinity(cachesRegistry.group(grp.groupId()), grp.affinity(), fut);
-                        }
+                        initAffinity(cachesRegistry.group(grp.groupId()), grp.affinity(), fut);
                     }
                 }
-            },
-            null);
+            }
+        );
     }
 
     /**
@@ -1228,7 +1259,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             .collect(Collectors.toList());
 
         try {
-            U.doInParallel(cctx.kernalContext().getSystemExecutorService(), affinityCaches, c, null);
+            U.doInParallel(cctx.kernalContext().getSystemExecutorService(), affinityCaches, c::applyx);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to execute affinity operation on cache groups", e);
@@ -1255,7 +1286,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         }
 
         try {
-            U.doInParallel(cctx.kernalContext().getSystemExecutorService(), affinityCaches, c, null);
+            U.doInParallel(cctx.kernalContext().getSystemExecutorService(), affinityCaches, c::applyx);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to execute affinity operation on cache groups", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 2e66e5b..3116d31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -1340,22 +1340,22 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         if (msgIdx != -1) {
             Map<Integer, IgniteBiInClosure[]> idxClsHandlers0 = msgHandlers.idxClsHandlers;
 
-            IgniteBiInClosure[] cacheClsHandlers = idxClsHandlers0.get(hndId);
+            IgniteBiInClosure[] cacheClsHandlers = idxClsHandlers0.compute(hndId, (key, clsHandlers) -> {
+                if (clsHandlers == null)
+                    clsHandlers = new IgniteBiInClosure[GridCacheMessage.MAX_CACHE_MSG_LOOKUP_INDEX];
 
-            if (cacheClsHandlers == null) {
-                cacheClsHandlers = new IgniteBiInClosure[GridCacheMessage.MAX_CACHE_MSG_LOOKUP_INDEX];
+                if(clsHandlers[msgIdx] != null)
+                    return null;
 
-                idxClsHandlers0.put(hndId, cacheClsHandlers);
-            }
+                clsHandlers[msgIdx] = c;
+
+                return clsHandlers;
+            });
 
-            if (cacheClsHandlers[msgIdx] != null)
+            if (cacheClsHandlers == null)
                 throw new IgniteException("Duplicate cache message ID found [hndId=" + hndId +
                     ", type=" + type + ']');
 
-            cacheClsHandlers[msgIdx] = c;
-
-            msgHandlers.idxClsHandlers = idxClsHandlers0;
-
             return;
         }
         else {
@@ -1572,7 +1572,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      */
     static class MessageHandlers {
         /** Indexed class handlers. */
-        volatile Map<Integer, IgniteBiInClosure[]> idxClsHandlers = new HashMap<>();
+        volatile Map<Integer, IgniteBiInClosure[]> idxClsHandlers = new ConcurrentHashMap<>();
 
         /** Handler registry. */
         ConcurrentMap<ListenerKey, IgniteBiInClosure<UUID, GridCacheMessage>>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 8b8efa3..b0e0d0d 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
@@ -109,6 +109,7 @@ import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -2622,7 +2623,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     err = e;
             }
             catch (Throwable e) {
-                err = e;
+                if (!(stop && X.hasCause(e, IgniteInterruptedCheckedException.class)))
+                    err = e;
             }
             finally {
                 if (err == null && !stop && !reconnectNeeded)

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 4a6bed4..59703c0 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import javax.management.MBeanServer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -68,6 +69,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.IgniteTransactionsEx;
 import org.apache.ignite.internal.binary.BinaryContext;
@@ -136,15 +138,16 @@ import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.F0;
+import org.apache.ignite.internal.util.InitializationProtector;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridPlainClosure;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -193,6 +196,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_C
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache;
+import static org.apache.ignite.internal.util.IgniteUtils.doInParallel;
 
 /**
  * Cache processor.
@@ -214,6 +218,10 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
     private final boolean walFsyncWithDedicatedWorker =
         IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER, false);
 
+    /** Enables start caches in parallel. */
+    private final boolean IGNITE_ALLOW_START_CACHES_IN_PARALLEL =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, true);
+
     /** Shared cache context. */
     private GridCacheSharedContext<?, ?> sharedCtx;
 
@@ -266,6 +274,9 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
     /** MBean group for cache group metrics */
     private final String CACHE_GRP_METRICS_MBEAN_GRP = "Cache groups";
 
+    /** Protector of initialization of specific value. */
+    private final InitializationProtector initializationProtector = new InitializationProtector();
+
     /**
      * @param ctx Kernal context.
      */
@@ -1285,71 +1296,6 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
     }
 
     /**
-     * @param cache Cache to start.
-     * @param schema Cache schema.
-     * @throws IgniteCheckedException If failed to start cache.
-     */
-    @SuppressWarnings({"TypeMayBeWeakened", "unchecked"})
-    private void startCache(GridCacheAdapter<?, ?> cache, QuerySchema schema) throws IgniteCheckedException {
-        GridCacheContext<?, ?> cacheCtx = cache.context();
-
-        CacheConfiguration cfg = cacheCtx.config();
-
-        // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set.
-        if (cfg.isStoreKeepBinary() && cfg.isStoreKeepBinary() != CacheConfiguration.DFLT_STORE_KEEP_BINARY
-            && !(ctx.config().getMarshaller() instanceof BinaryMarshaller))
-            U.warn(log, "CacheConfiguration.isStoreKeepBinary() configuration property will be ignored because " +
-                "BinaryMarshaller is not used");
-
-        // Start managers.
-        for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
-            mgr.start(cacheCtx);
-
-        cacheCtx.initConflictResolver();
-
-        if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
-            GridCacheContext<?, ?> dhtCtx = cacheCtx.near().dht().context();
-
-            // Start DHT managers.
-            for (GridCacheManager mgr : dhtManagers(dhtCtx))
-                mgr.start(dhtCtx);
-
-            dhtCtx.initConflictResolver();
-
-            // Start DHT cache.
-            dhtCtx.cache().start();
-
-            if (log.isDebugEnabled())
-                log.debug("Started DHT cache: " + dhtCtx.cache().name());
-        }
-
-        ctx.continuous().onCacheStart(cacheCtx);
-
-        cacheCtx.cache().start();
-
-        ctx.query().onCacheStart(cacheCtx, schema);
-
-        cacheCtx.onStarted();
-
-        String memPlcName = cfg.getDataRegionName();
-
-        if (memPlcName == null && ctx.config().getDataStorageConfiguration() != null)
-            memPlcName = ctx.config().getDataStorageConfiguration().getDefaultDataRegionConfiguration().getName();
-
-        if (log.isInfoEnabled()) {
-            log.info("Started cache [name=" + cfg.getName() +
-                ", id=" + cacheCtx.cacheId() +
-                (cfg.getGroupName() != null ? ", group=" + cfg.getGroupName() : "") +
-                ", memoryPolicyName=" + memPlcName +
-                ", mode=" + cfg.getCacheMode() +
-                ", atomicity=" + cfg.getAtomicityMode() +
-                ", backups=" + cfg.getBackups() +
-                ", mvcc=" + cacheCtx.mvccEnabled() +']' +
-                ", encryptionEnabled=" + cfg.isEncryptionEnabled() +']');
-        }
-    }
-
-    /**
      * @param cache Cache to stop.
      * @param cancel Cancel flag.
      * @param destroy Destroy data flag. Setting to <code>true</code> will remove all cache data.
@@ -1600,7 +1546,13 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
         GridCacheDrManager drMgr = pluginMgr.createComponent(GridCacheDrManager.class);
         CacheStoreManager storeMgr = pluginMgr.createComponent(CacheStoreManager.class);
 
-        storeMgr.initialize(cfgStore, sesHolders);
+        if (cfgStore == null)
+            storeMgr.initialize(cfgStore, sesHolders);
+        else
+            initializationProtector.protect(
+                cfgStore,
+                () -> storeMgr.initialize(cfgStore, sesHolders)
+            );
 
         GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
             ctx,
@@ -2017,18 +1969,13 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
         IgniteInternalFuture<?> res = sharedCtx.affinity().initCachesOnLocalJoin(
             locJoinCtx.cacheGroupDescriptors(), locJoinCtx.cacheDescriptors());
 
-        for (T2<DynamicCacheDescriptor, NearCacheConfiguration> t : locJoinCtx.caches()) {
-            DynamicCacheDescriptor desc = t.get1();
+        List<StartCacheInfo> startCacheInfos = locJoinCtx.caches().stream()
+            .map(cacheInfo -> new StartCacheInfo(cacheInfo.get1(), cacheInfo.get2(), exchTopVer, false))
+            .collect(Collectors.toList());
 
-            prepareCacheStart(
-                desc.cacheConfiguration(),
-                desc,
-                t.get2(),
-                exchTopVer,
-                false);
+        prepareStartCaches(startCacheInfos);
 
-            context().exchange().exchangerUpdateHeartbeat();
-        }
+        context().exchange().exchangerUpdateHeartbeat();
 
         if (log.isInfoEnabled())
             log.info("Starting caches on local join performed in " + (System.currentTimeMillis() - time) + " ms.");
@@ -2054,24 +2001,164 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
      */
     public Collection<DynamicCacheDescriptor> startReceivedCaches(UUID nodeId, AffinityTopologyVersion exchTopVer)
         throws IgniteCheckedException {
-        List<DynamicCacheDescriptor> started = cachesInfo.cachesReceivedFromJoin(nodeId);
+        List<DynamicCacheDescriptor> receivedCaches = cachesInfo.cachesReceivedFromJoin(nodeId);
 
-        for (DynamicCacheDescriptor desc : started) {
-            IgnitePredicate<ClusterNode> filter = desc.groupDescriptor().config().getNodeFilter();
+        List<StartCacheInfo> startCacheInfos = receivedCaches.stream()
+            .filter(desc -> isLocalAffinity(desc.groupDescriptor().config()))
+            .map(desc -> new StartCacheInfo(desc, null, exchTopVer, false))
+            .collect(Collectors.toList());
 
-            if (CU.affinityNode(ctx.discovery().localNode(), filter)) {
-                prepareCacheStart(
-                    desc.cacheConfiguration(),
-                    desc,
-                    null,
-                    exchTopVer,
-                    false);
+        prepareStartCaches(startCacheInfos);
+
+        return receivedCaches;
+    }
+
+    /**
+     * @param cacheConfiguration Checked configuration.
+     * @return {@code true} if local node is affinity node for cache.
+     */
+    private boolean isLocalAffinity(CacheConfiguration cacheConfiguration) {
+        return CU.affinityNode(ctx.discovery().localNode(), cacheConfiguration.getNodeFilter());
+    }
+
+    /**
+     * Start all input caches in parallel.
+     *
+     * @param startCacheInfos All caches information for start.
+     */
+    void prepareStartCaches(Collection<StartCacheInfo> startCacheInfos) throws IgniteCheckedException {
+        prepareStartCaches(startCacheInfos, (data, operation) -> {
+            operation.accept(data);// PROXY
+        });
+    }
+
+    /**
+     * Trying to start all input caches in parallel and skip failed caches.
+     *
+     * @param startCacheInfos Caches info for start.
+     * @return Caches which was failed.
+     * @throws IgniteCheckedException if failed.
+     */
+    Map<StartCacheInfo, IgniteCheckedException> prepareStartCachesIfPossible(Collection<StartCacheInfo> startCacheInfos) throws IgniteCheckedException {
+        HashMap<StartCacheInfo, IgniteCheckedException> failedCaches = new HashMap<>();
+
+        prepareStartCaches(startCacheInfos, (data, operation) -> {
+            try {
+                operation.accept(data);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw e;
+            }
+            catch (IgniteCheckedException e) {
+                log.warning("Cache can not be started : cache=" + data.getStartedConfiguration().getName());
+
+                failedCaches.put(data, e);
+            }
+        });
+
+        return failedCaches;
+    }
+
+    /**
+     * Start all input caches in parallel.
+     *
+     * @param startCacheInfos All caches information for start.
+     * @param cacheStartFailHandler Fail handler for one cache start.
+     */
+    private void prepareStartCaches(
+        Collection<StartCacheInfo> startCacheInfos,
+        StartCacheFailHandler<StartCacheInfo> cacheStartFailHandler
+    ) throws IgniteCheckedException {
+        if (!IGNITE_ALLOW_START_CACHES_IN_PARALLEL || startCacheInfos.size() <= 1) {
+            for (StartCacheInfo startCacheInfo : startCacheInfos) {
+                cacheStartFailHandler.handle(
+                    startCacheInfo,
+                    cacheInfo -> prepareCacheStart(
+                        cacheInfo.getCacheDescriptor().cacheConfiguration(),
+                        cacheInfo.getCacheDescriptor(),
+                        cacheInfo.getReqNearCfg(),
+                        cacheInfo.getExchangeTopVer(),
+                        cacheInfo.isDisabledAfterStart()
+                    )
+                );
 
                 context().exchange().exchangerUpdateHeartbeat();
             }
         }
+        else {
+            Map<StartCacheInfo, GridCacheContext> cacheContexts = new ConcurrentHashMap<>();
+
+            int parallelismLvl = sharedCtx.kernalContext().config().getSystemThreadPoolSize();
+
+            // Reserve at least 2 threads for system operations.
+            parallelismLvl = Math.max(1, parallelismLvl - 2);
+
+            doInParallel(
+                parallelismLvl,
+                sharedCtx.kernalContext().getSystemExecutorService(),
+                startCacheInfos,
+                startCacheInfo ->
+                    cacheStartFailHandler.handle(
+                        startCacheInfo,
+                        cacheInfo -> {
+                            GridCacheContext cacheCtx = prepareCacheContext(
+                                cacheInfo.getCacheDescriptor().cacheConfiguration(),
+                                cacheInfo.getCacheDescriptor(),
+                                cacheInfo.getReqNearCfg(),
+                                cacheInfo.getExchangeTopVer(),
+                                cacheInfo.isDisabledAfterStart()
+                            );
+                            cacheContexts.put(cacheInfo, cacheCtx);
+
+                            context().exchange().exchangerUpdateHeartbeat();
+                        }
+                    )
+            );
 
-        return started;
+            /*
+             * This hack required because we can't start sql schema in parallel by folowing reasons:
+             * * checking index to duplicate(and other checking) require one order on every nodes.
+             * * onCacheStart and createSchema contains a lot of mutex.
+             *
+             * TODO IGNITE-9729
+             */
+            Set<StartCacheInfo> successfullyPreparedCaches = cacheContexts.keySet();
+
+            List<StartCacheInfo> cacheInfosInOriginalOrder = startCacheInfos.stream()
+                .filter(successfullyPreparedCaches::contains)
+                .collect(Collectors.toList());
+
+            for (StartCacheInfo startCacheInfo : cacheInfosInOriginalOrder) {
+                cacheStartFailHandler.handle(
+                    startCacheInfo,
+                    cacheInfo -> {
+                        ctx.query().onCacheStart(
+                            cacheContexts.get(cacheInfo),
+                            cacheInfo.getCacheDescriptor().schema() != null
+                                ? cacheInfo.getCacheDescriptor().schema()
+                                : new QuerySchema()
+                        );
+
+                        context().exchange().exchangerUpdateHeartbeat();
+                    }
+                );
+            }
+
+            doInParallel(
+                parallelismLvl,
+                sharedCtx.kernalContext().getSystemExecutorService(),
+                cacheContexts.entrySet(),
+                cacheCtxEntry ->
+                    cacheStartFailHandler.handle(
+                        cacheCtxEntry.getKey(),
+                        cacheInfo -> {
+                            onCacheStarted(cacheCtxEntry.getValue());
+
+                            context().exchange().exchangerUpdateHeartbeat();
+                        }
+                    )
+            );
+        }
     }
 
     /**
@@ -2090,71 +2177,156 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
         AffinityTopologyVersion exchTopVer,
         boolean disabledAfterStart
     ) throws IgniteCheckedException {
+        GridCacheContext cacheCtx = prepareCacheContext(startCfg, desc, reqNearCfg, exchTopVer, disabledAfterStart);
+
+        ctx.query().onCacheStart(cacheCtx, desc.schema() != null ? desc.schema() : new QuerySchema());
+
+        onCacheStarted(cacheCtx);
+    }
+
+    /**
+     * Preparing cache context to start.
+     *
+     * @param startCfg Cache configuration to use.
+     * @param desc Cache descriptor.
+     * @param reqNearCfg Near configuration if specified for client cache start request.
+     * @param exchTopVer Current exchange version.
+     * @param disabledAfterStart If true, then we will discard restarting state from proxies. If false then we will change
+     *  state of proxies to restarting
+     * @return Created {@link GridCacheContext}.
+     * @throws IgniteCheckedException if failed.
+     */
+    private GridCacheContext prepareCacheContext(
+        CacheConfiguration startCfg,
+        DynamicCacheDescriptor desc,
+        @Nullable NearCacheConfiguration reqNearCfg,
+        AffinityTopologyVersion exchTopVer,
+        boolean disabledAfterStart
+    ) throws IgniteCheckedException {
         assert !caches.containsKey(startCfg.getName()) : startCfg.getName();
 
         CacheConfiguration ccfg = new CacheConfiguration(startCfg);
 
         CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-        boolean affNode;
+        boolean affNode = checkForAffinityNode(desc, reqNearCfg, ccfg);
 
-        if (ccfg.getCacheMode() == LOCAL) {
-            affNode = true;
+        preparePageStore(desc, affNode);
+
+        CacheGroupContext grp = prepareCacheGroup(desc, exchTopVer, cacheObjCtx, affNode, startCfg.getGroupName());
+
+        GridCacheContext cacheCtx = createCache(ccfg,
+            grp,
+            null,
+            desc,
+            exchTopVer,
+            cacheObjCtx,
+            affNode,
+            true,
+            disabledAfterStart
+        );
 
+        initCacheContext(cacheCtx, ccfg, desc.deploymentId());
+
+        return cacheCtx;
+    }
+
+    /**
+     * Check for affinity node and customize near configuration if needed.
+     *
+     * @param desc Cache descriptor.
+     * @param reqNearCfg Near configuration if specified for client cache start request.
+     * @param ccfg Cache configuration to use.
+     * @return {@code true} if it is affinity node for cache.
+     */
+    private boolean checkForAffinityNode(
+        DynamicCacheDescriptor desc,
+        @Nullable NearCacheConfiguration reqNearCfg,
+        CacheConfiguration ccfg
+    ) {
+        if (ccfg.getCacheMode() == LOCAL) {
             ccfg.setNearConfiguration(null);
-        }
-        else if (CU.affinityNode(ctx.discovery().localNode(), desc.groupDescriptor().config().getNodeFilter()))
-            affNode = true;
-        else {
-            affNode = false;
 
-            ccfg.setNearConfiguration(reqNearCfg);
+            return true;
         }
 
-        if (sharedCtx.pageStore() != null && affNode)
-            sharedCtx.pageStore().initializeForCache(desc.groupDescriptor(), desc.toStoredData());
-
-        String grpName = startCfg.getGroupName();
+        if (isLocalAffinity(desc.groupDescriptor().config()))
+            return true;
 
-        CacheGroupContext grp = null;
+        ccfg.setNearConfiguration(reqNearCfg);
 
-        if (grpName != null) {
-            for (CacheGroupContext grp0 : cacheGrps.values()) {
-                if (grp0.sharedGroup() && grpName.equals(grp0.name())) {
-                    grp = grp0;
+        return false;
+    }
 
-                    break;
-                }
-            }
+    /**
+     * Prepare page store for start cache.
+     *
+     * @param desc Cache descriptor.
+     * @param affNode {@code true} if it is affinity node for cache.
+     * @throws IgniteCheckedException if failed.
+     */
+    private void preparePageStore(DynamicCacheDescriptor desc, boolean affNode) throws IgniteCheckedException {
+        if (sharedCtx.pageStore() != null && affNode)
+            initializationProtector.protect(
+                desc.groupDescriptor().groupId(),
+                () -> sharedCtx.pageStore().initializeForCache(desc.groupDescriptor(), desc.toStoredData())
+            );
+    }
 
-            if (grp == null) {
-                grp = startCacheGroup(desc.groupDescriptor(),
+    /**
+     * Prepare cache group to start cache.
+     *
+     * @param desc Cache descriptor.
+     * @param exchTopVer Current exchange version.
+     * @param cacheObjCtx Cache object context.
+     * @param affNode {@code true} if it is affinity node for cache.
+     * @param grpName Group name.
+     * @return Prepared cache group context.
+     * @throws IgniteCheckedException if failed.
+     */
+    private CacheGroupContext prepareCacheGroup(
+        DynamicCacheDescriptor desc,
+        AffinityTopologyVersion exchTopVer,
+        CacheObjectContext cacheObjCtx,
+        boolean affNode,
+        String grpName
+    ) throws IgniteCheckedException {
+        if (grpName != null) {
+            return initializationProtector.protect(
+                desc.groupId(),
+                () -> findCacheGroup(grpName),
+                () -> startCacheGroup(
+                    desc.groupDescriptor(),
                     desc.cacheType(),
                     affNode,
                     cacheObjCtx,
-                    exchTopVer);
-            }
-        }
-        else {
-            grp = startCacheGroup(desc.groupDescriptor(),
-                desc.cacheType(),
-                affNode,
-                cacheObjCtx,
-                exchTopVer);
+                    exchTopVer
+                )
+            );
         }
 
-        GridCacheContext cacheCtx = createCache(ccfg,
-            grp,
-            null,
-            desc,
-            exchTopVer,
-            cacheObjCtx,
+        return startCacheGroup(desc.groupDescriptor(),
+            desc.cacheType(),
             affNode,
-            true,
-            disabledAfterStart
+            cacheObjCtx,
+            exchTopVer
         );
+    }
 
-        cacheCtx.dynamicDeploymentId(desc.deploymentId());
+    /**
+     * Initialize created cache context.
+     *
+     * @param cacheCtx Cache context to initializtion.
+     * @param cfg Cache configuration.
+     * @param deploymentId Dynamic deployment ID.
+     * @throws IgniteCheckedException if failed.
+     */
+    private void initCacheContext(
+        GridCacheContext<?, ?> cacheCtx,
+        CacheConfiguration cfg,
+        IgniteUuid deploymentId
+    ) throws IgniteCheckedException {
+        cacheCtx.dynamicDeploymentId(deploymentId);
 
         GridCacheAdapter cache = cacheCtx.cache();
 
@@ -2162,7 +2334,67 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
 
         caches.put(cacheCtx.name(), cache);
 
-        startCache(cache, desc.schema() != null ? desc.schema() : new QuerySchema());
+        // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set.
+        if (cfg.isStoreKeepBinary() && cfg.isStoreKeepBinary() != CacheConfiguration.DFLT_STORE_KEEP_BINARY
+            && !(ctx.config().getMarshaller() instanceof BinaryMarshaller))
+            U.warn(log, "CacheConfiguration.isStoreKeepBinary() configuration property will be ignored because " +
+                "BinaryMarshaller is not used");
+
+        // Start managers.
+        for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
+            mgr.start(cacheCtx);
+
+        cacheCtx.initConflictResolver();
+
+        if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
+            GridCacheContext<?, ?> dhtCtx = cacheCtx.near().dht().context();
+
+            // Start DHT managers.
+            for (GridCacheManager mgr : dhtManagers(dhtCtx))
+                mgr.start(dhtCtx);
+
+            dhtCtx.initConflictResolver();
+
+            // Start DHT cache.
+            dhtCtx.cache().start();
+
+            if (log.isDebugEnabled())
+                log.debug("Started DHT cache: " + dhtCtx.cache().name());
+        }
+
+        ctx.continuous().onCacheStart(cacheCtx);
+
+        cacheCtx.cache().start();
+    }
+
+    /**
+     * Handle of cache context which was fully prepared.
+     *
+     * @param cacheCtx Fully prepared context.
+     * @throws IgniteCheckedException if failed.
+     */
+    private void onCacheStarted(GridCacheContext cacheCtx) throws IgniteCheckedException {
+        GridCacheAdapter cache = cacheCtx.cache();
+        CacheConfiguration cfg = cacheCtx.config();
+        CacheGroupContext grp = cacheGrps.get(cacheCtx.groupId());
+
+        cacheCtx.onStarted();
+
+        String dataRegion = cfg.getDataRegionName();
+
+        if (dataRegion == null && ctx.config().getDataStorageConfiguration() != null)
+            dataRegion = ctx.config().getDataStorageConfiguration().getDefaultDataRegionConfiguration().getName();
+
+        if (log.isInfoEnabled()) {
+            log.info("Started cache [name=" + cfg.getName() +
+                ", id=" + cacheCtx.cacheId() +
+                (cfg.getGroupName() != null ? ", group=" + cfg.getGroupName() : "") +
+                ", dataRegionName=" + dataRegion +
+                ", mode=" + cfg.getCacheMode() +
+                ", atomicity=" + cfg.getAtomicityMode() +
+                ", backups=" + cfg.getBackups() +
+                ", mvcc=" + cacheCtx.mvccEnabled() + ']');
+        }
 
         grp.onCacheStarted(cacheCtx);
 
@@ -2170,6 +2402,17 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
     }
 
     /**
+     * @param grpName Group name.
+     * @return Found group or null.
+     */
+    private CacheGroupContext findCacheGroup(String grpName) {
+        return cacheGrps.values().stream()
+            .filter(grp -> grp.sharedGroup() && grpName.equals(grp.name()))
+            .findAny()
+            .orElse(null);
+    }
+
+    /**
      * Restarts proxies of caches if they was marked as restarting. Requires external synchronization - shouldn't be
      * called concurrently with another caches restart.
      */
@@ -4839,7 +5082,7 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
 
                     // Check if we were asked to start a near cache.
                     if (nearCfg != null) {
-                        if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) {
+                        if (isLocalAffinity(descCfg)) {
                             // If we are on a data node and near cache was enabled, return success, else - fail.
                             if (descCfg.getNearConfiguration() != null)
                                 return null;
@@ -4851,7 +5094,7 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
                             // If local node has near cache, return success.
                             req.clientStartOnly(true);
                     }
-                    else if (!CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter()))
+                    else if (!isLocalAffinity(descCfg))
                         req.clientStartOnly(true);
 
                     req.deploymentId(desc.deploymentId());
@@ -5020,6 +5263,22 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
     }
 
     /**
+     * Handle of fail during cache start.
+     *
+     * @param <T> Type of started data.
+     */
+    private static interface StartCacheFailHandler<T> {
+        /**
+         * Handle of fail.
+         *
+         * @param data Start data.
+         * @param startCacheOperation Operation for start cache.
+         * @throws IgniteCheckedException if failed.
+         */
+        void handle(T data, IgniteThrowableConsumer<T> startCacheOperation) throws IgniteCheckedException;
+    }
+
+    /**
      *
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StartCacheInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StartCacheInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StartCacheInfo.java
new file mode 100644
index 0000000..a5aea26
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StartCacheInfo.java
@@ -0,0 +1,113 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Specific cache information for start.
+ */
+public class StartCacheInfo {
+    /** Cache configuration for start. */
+    private final CacheConfiguration startedConf;
+
+    /** Cache descriptor for start. */
+    private final DynamicCacheDescriptor desc;
+
+    /** Near cache configuration for start. */
+    private final @Nullable NearCacheConfiguration reqNearCfg;
+
+    /** Exchange topology version in which starting happened. */
+    private final AffinityTopologyVersion exchTopVer;
+
+    /** Disable started cache after start or not. */
+    private final boolean disabledAfterStart;
+
+    /**
+     * @param desc Cache configuration for start.
+     * @param reqNearCfg Near cache configuration for start.
+     * @param exchTopVer Exchange topology version in which starting happened.
+     * @param disabledAfterStart Disable started cache after start or not.
+     */
+    public StartCacheInfo(DynamicCacheDescriptor desc,
+        NearCacheConfiguration reqNearCfg,
+        AffinityTopologyVersion exchTopVer, boolean disabledAfterStart) {
+        this(desc.cacheConfiguration(), desc, reqNearCfg, exchTopVer, disabledAfterStart);
+    }
+
+    /**
+     * @param conf Cache configuration for start.
+     * @param desc Cache descriptor for start.
+     * @param reqNearCfg Near cache configuration for start.
+     * @param exchTopVer Exchange topology version in which starting happened.
+     * @param disabledAfterStart Disable started cache after start or not.
+     */
+    public StartCacheInfo(CacheConfiguration conf, DynamicCacheDescriptor desc,
+        NearCacheConfiguration reqNearCfg,
+        AffinityTopologyVersion exchTopVer, boolean disabledAfterStart) {
+        startedConf = conf;
+        this.desc = desc;
+        this.reqNearCfg = reqNearCfg;
+        this.exchTopVer = exchTopVer;
+        this.disabledAfterStart = disabledAfterStart;
+    }
+
+    /**
+     * @return Cache configuration for start.
+     */
+    public CacheConfiguration getStartedConfiguration() {
+        return startedConf;
+    }
+
+    /**
+     * @return Cache descriptor for start.
+     */
+    public DynamicCacheDescriptor getCacheDescriptor() {
+        return desc;
+    }
+
+    /**
+     * @return Near cache configuration for start.
+     */
+    @Nullable public NearCacheConfiguration getReqNearCfg() {
+        return reqNearCfg;
+    }
+
+    /**
+     * @return Exchange topology version in which starting happened.
+     */
+    public AffinityTopologyVersion getExchangeTopVer() {
+        return exchTopVer;
+    }
+
+    /**
+     * @return Disable started cache after start or not.
+     */
+    public boolean isDisabledAfterStart() {
+        return disabledAfterStart;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StartCacheInfo.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 d494857..0c2cbe2 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
@@ -3512,36 +3512,34 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             U.doInParallel(
                 cctx.kernalContext().getSystemExecutorService(),
                 nonLocalCacheGroupDescriptors(),
-                new IgniteInClosureX<CacheGroupDescriptor>() {
-                    @Override public void applyx(CacheGroupDescriptor grpDesc) {
-                        CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpDesc.groupId());
-
-                        GridDhtPartitionTopology top = grpCtx != null
-                            ? grpCtx.topology()
-                            : cctx.exchange().clientTopology(grpDesc.groupId(), events().discoveryCache());
-
-                        // Do not validate read or write through caches or caches with disabled rebalance
-                        // or ExpiryPolicy is set or validation is disabled.
-                        if (grpCtx == null
-                            || grpCtx.config().isReadThrough()
-                            || grpCtx.config().isWriteThrough()
-                            || grpCtx.config().getCacheStoreFactory() != null
-                            || grpCtx.config().getRebalanceDelay() == -1
-                            || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE
-                            || grpCtx.config().getExpiryPolicyFactory() == null
-                            || SKIP_PARTITION_SIZE_VALIDATION)
-                            return;
+                grpDesc -> {
+                    CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpDesc.groupId());
+
+                    GridDhtPartitionTopology top = grpCtx != null
+                        ? grpCtx.topology()
+                        : cctx.exchange().clientTopology(grpDesc.groupId(), events().discoveryCache());
+
+                    // Do not validate read or write through caches or caches with disabled rebalance
+                    // or ExpiryPolicy is set or validation is disabled.
+                    if (grpCtx == null
+                        || grpCtx.config().isReadThrough()
+                        || grpCtx.config().isWriteThrough()
+                        || grpCtx.config().getCacheStoreFactory() != null
+                        || grpCtx.config().getRebalanceDelay() == -1
+                        || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE
+                        || grpCtx.config().getExpiryPolicyFactory() == null
+                        || SKIP_PARTITION_SIZE_VALIDATION)
+                        return;
 
-                        try {
-                            validator.validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture.this, top, msgs);
-                        }
-                        catch (IgniteCheckedException ex) {
-                            log.warning("Partition states validation has failed for group: " + grpCtx.cacheOrGroupName() + ". " + ex.getMessage());
-                            // TODO: Handle such errors https://issues.apache.org/jira/browse/IGNITE-7833
-                        }
+                    try {
+                        validator.validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture.this, top, msgs);
                     }
-                },
-                null);
+                    catch (IgniteCheckedException ex) {
+                        log.warning("Partition states validation has failed for group: " + grpCtx.cacheOrGroupName() + ". " + ex.getMessage());
+                        // TODO: Handle such errors https://issues.apache.org/jira/browse/IGNITE-7833
+                    }
+                }
+            );
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to validate partitions state", e);
@@ -3561,21 +3559,19 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             U.doInParallel(
                 cctx.kernalContext().getSystemExecutorService(),
                 nonLocalCacheGroupDescriptors(),
-                new IgniteInClosureX<CacheGroupDescriptor>() {
-                    @Override public void applyx(CacheGroupDescriptor grpDesc) {
-                        CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpDesc.groupId());
+                grpDesc -> {
+                    CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpDesc.groupId());
 
-                        GridDhtPartitionTopology top = grpCtx != null
-                            ? grpCtx.topology()
-                            : cctx.exchange().clientTopology(grpDesc.groupId(), events().discoveryCache());
+                    GridDhtPartitionTopology top = grpCtx != null
+                        ? grpCtx.topology()
+                        : cctx.exchange().clientTopology(grpDesc.groupId(), events().discoveryCache());
 
-                        if (!CU.isPersistentCache(grpDesc.config(), cctx.gridConfig().getDataStorageConfiguration()))
-                            assignPartitionSizes(top);
-                        else
-                            assignPartitionStates(top);
-                    }
-                },
-                null);
+                    if (!CU.isPersistentCache(grpDesc.config(), cctx.gridConfig().getDataStorageConfiguration()))
+                        assignPartitionSizes(top);
+                    else
+                        assignPartitionStates(top);
+                }
+            );
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to assign partition states", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 e6f374a..2fe0eb8 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
@@ -142,6 +142,7 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 import java.util.zip.ZipInputStream;
@@ -216,19 +217,16 @@ import org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryNativeLoader;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.GridPeerDeployAware;
 import org.apache.ignite.internal.util.lang.GridTuple;
-import org.apache.ignite.internal.util.lang.IgniteInClosureX;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFutureCancelledException;
@@ -236,6 +234,7 @@ import org.apache.ignite.lang.IgniteFutureTimeoutException;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.marshaller.Marshaller;
@@ -10739,54 +10738,91 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Execute operation on data in parallel.
+     *
      * @param executorSvc Service for parallel execution.
      * @param srcDatas List of data for parallelization.
-     * @param consumer Logic for execution of on each item of data.
-     * @param errHnd Optionan error handler. If not {@code null}, an error of each item execution will be passed to
-     *      this handler. If error handler is not {@code null}, the exception will not be thrown from this method.
+     * @param operation Logic for execution of on each item of data.
      * @param <T> Type of data.
-     * @return List of (item, execution future) tuples.
-     * @throws IgniteCheckedException If parallel execution failed and {@code errHnd} is {@code null}.
+     * @throws IgniteCheckedException if parallel execution was failed.
      */
-    public static <T> List<T2<T, Future<Object>>> doInParallel(
+    public static <T> void doInParallel(ExecutorService executorSvc, Collection<T> srcDatas,
+        IgniteThrowableConsumer<T> operation) throws IgniteCheckedException, IgniteInterruptedCheckedException {
+        doInParallel(srcDatas.size(), executorSvc, srcDatas, operation);
+    }
+
+    /**
+     * Execute operation on data in parallel.
+     *
+     * @param parallelismLvl Number of threads on which it should be executed.
+     * @param executorSvc Service for parallel execution.
+     * @param srcDatas List of data for parallelization.
+     * @param operation Logic for execution of on each item of data.
+     * @param <T> Type of data.
+     * @throws IgniteCheckedException if parallel execution was failed.
+     */
+    public static <T> void doInParallel(
+        int parallelismLvl,
         ExecutorService executorSvc,
         Collection<T> srcDatas,
-        IgniteInClosureX<T> consumer,
-        @Nullable IgniteBiInClosure<T, Throwable> errHnd
-    ) throws IgniteCheckedException {
-        List<T2<T, Future<Object>>> consumerFutures = srcDatas.stream()
-            .map(item -> new T2<>(
-                item,
-                executorSvc.submit(() -> {
-                    consumer.apply(item);
+        IgniteThrowableConsumer<T> operation
+    ) throws IgniteCheckedException, IgniteInterruptedCheckedException {
+        List<List<T>> batches = IntStream.range(0, parallelismLvl)
+            .mapToObj(i -> new ArrayList<T>())
+            .collect(Collectors.toList());
 
-                    return null;
-                })))
+        int i = 0;
+
+        for (T src : srcDatas)
+            batches.get(i++ % parallelismLvl).add(src);
+
+        List<Future<Object>> consumerFutures = batches.stream()
+            .filter(batch -> !batch.isEmpty())
+            .map(batch -> executorSvc.submit(() -> {
+                for (T item : batch)
+                    operation.accept(item);
+
+                return null;
+            }))
             .collect(Collectors.toList());
 
-        IgniteCheckedException composite = null;
+        Throwable error =null;
 
-        for (T2<T, Future<Object>> tup : consumerFutures) {
+        for (Future<Object> future : consumerFutures) {
             try {
-                getUninterruptibly(tup.get2());
+                future.get();
             }
-            catch (ExecutionException e) {
-                if (errHnd != null)
-                    errHnd.apply(tup.get1(), e.getCause());
-                else {
-                    if (composite == null)
-                        composite = new IgniteCheckedException("Failed to execute one of the tasks " +
-                            "(see suppressed exception for details)");
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
 
-                    composite.addSuppressed(e.getCause());
-                }
+                throw new IgniteInterruptedCheckedException(e);
+            }
+            catch (ExecutionException e) {
+                if(error == null)
+                    error = e.getCause();
+                else
+                    error.addSuppressed(e.getCause());
+            }
+            catch (CancellationException e) {
+                if(error == null)
+                    error = e;
+                else
+                    error.addSuppressed(e);
             }
         }
 
-        if (composite != null)
-            throw composite;
+        if (error != null) {
+            if (error instanceof IgniteCheckedException)
+                throw (IgniteCheckedException)error;
 
-        return consumerFutures;
+            if (error instanceof RuntimeException)
+                throw (RuntimeException)error;
+
+            if (error instanceof Error)
+                throw (Error)error;
+
+            throw new IgniteCheckedException(error);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/util/InitializationProtector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/InitializationProtector.java b/modules/core/src/main/java/org/apache/ignite/internal/util/InitializationProtector.java
new file mode 100644
index 0000000..7c501c4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/InitializationProtector.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.util;
+
+import java.util.concurrent.locks.Lock;
+import java.util.function.Supplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.IgniteThrowableRunner;
+
+/**
+ * Class for avoid multiple initialization of specific value from various threads.
+ */
+public class InitializationProtector {
+    /** Default striped lock concurrency level. */
+    private static final int DEFAULT_CONCURRENCY_LEVEL = Runtime.getRuntime().availableProcessors();
+
+    /** Striped lock. */
+    private GridStripedLock stripedLock = new GridStripedLock(DEFAULT_CONCURRENCY_LEVEL);
+
+    /**
+     * @param protectedKey Unique value by which initialization code should be run only one time.
+     * @param initializedVal Supplier for given already initialized value if it exist or null as sign that
+     * initialization required.
+     * @param initializationCode Code for initialization value corresponding protectedKey. Should be idempotent.
+     * @param <T> Type of initialization value.
+     * @return Initialized value.
+     * @throws IgniteCheckedException if initialization was failed.
+     */
+    public <T> T protect(Object protectedKey, Supplier<T> initializedVal,
+        IgniteThrowableRunner initializationCode) throws IgniteCheckedException {
+        T value = initializedVal.get();
+
+        if (value != null)
+            return value;
+
+        Lock lock = stripedLock.getLock(protectedKey.hashCode() % stripedLock.concurrencyLevel());
+
+        lock.lock();
+        try {
+            value = initializedVal.get();
+
+            if (value != null)
+                return value;
+
+            initializationCode.run();
+
+            return initializedVal.get();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * It method allows to avoid simultaneous initialization from various threads.
+     *
+     * @param protectedKey Unique value by which initialization code should be run only from one thread in one time.
+     * @param initializationCode Code for initialization value corresponding protectedKey. Should be idempotent.
+     * @throws IgniteCheckedException if initialization was failed.
+     */
+    public void protect(Object protectedKey, IgniteThrowableRunner initializationCode) throws IgniteCheckedException {
+        protect(protectedKey, () -> null, initializationCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableConsumer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableConsumer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableConsumer.java
new file mode 100644
index 0000000..46813a9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableConsumer.java
@@ -0,0 +1,37 @@
+/*
+ * 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.lang;
+
+import java.io.Serializable;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Represents an operation that accepts a single input argument and returns no result. Unlike most other functional
+ * interfaces, {@code IgniteThrowableConsumer} is expected to operate via side-effects.
+ *
+ * @param <E> Type of closure parameter.
+ */
+public interface IgniteThrowableConsumer<E> extends Serializable {
+    /**
+     * Consumer body.
+     *
+     * @param e Consumer parameter.
+     * @throws IgniteCheckedException if body execution was failed.
+     */
+    public void accept(E e) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableRunner.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableRunner.java
new file mode 100644
index 0000000..a5c95e1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableRunner.java
@@ -0,0 +1,30 @@
+/*
+ * 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.lang;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Represents a throwable runner.
+ */
+public interface IgniteThrowableRunner {
+    /**
+     * Execute a body.
+     */
+    void run() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java
new file mode 100644
index 0000000..4e30d1c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheStartInParallelTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.cache.distributed;
+
+import java.util.ArrayList;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class CacheStartInParallelTest extends GridCommonAbstractTest {
+    /** */
+    private static final int CACHES_COUNT = 40;
+
+    /** */
+    private static final String STATIC_CACHE_PREFIX = "static-cache-";
+
+    /** */
+    private static final String DYNAMIC_CACHE_PREFIX = "dynamic-cache-";
+
+    /** */
+    private static boolean isStaticCache = true;
+
+    /** */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setSystemThreadPoolSize(10);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        long sz = 100 * 1024 * 1024;
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration().setPageSize(1024)
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration().setPersistenceEnabled(true).setInitialSize(sz).setMaxSize(sz))
+            .setWalMode(WALMode.LOG_ONLY).setCheckpointFrequency(24L * 60 * 60 * 1000);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        if (isStaticCache) {
+            ArrayList<Object> staticCaches = new ArrayList<>(CACHES_COUNT);
+
+            for (int i = 0; i < CACHES_COUNT; i++)
+                staticCaches.add(cacheConfiguration(STATIC_CACHE_PREFIX + i));
+
+            cfg.setCacheConfiguration(staticCaches.toArray(new CacheConfiguration[CACHES_COUNT]));
+        }
+
+        return cfg;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration(String cacheName) {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(cacheName);
+        cfg.setBackups(1);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cleanupTestData();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        cleanupTestData();
+    }
+
+    /** */
+    private void cleanupTestData() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        System.clearProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL);
+
+        isStaticCache = true;
+    }
+
+    /**
+     * Checking that start static caches in parallel faster than consistenly.
+     *
+     * @throws Exception if fail.
+     */
+    public void testParallelizationAcceleratesStartOfStaticCaches() throws Exception {
+        //start caches consistently.
+        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "false");
+
+        long startTime = System.currentTimeMillis();
+
+        IgniteEx igniteEx = startGrid(0);
+
+        igniteEx.cluster().active(true);
+
+        long totalStartTimeConsistently = System.currentTimeMillis() - startTime;
+
+        //check cache started.
+        for (int i = 0; i < CACHES_COUNT; i++)
+            igniteEx.cache(STATIC_CACHE_PREFIX + i).put(i, i);
+
+        stopAllGrids();
+
+        //start caches in parallel.
+        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "true");
+
+        startTime = System.currentTimeMillis();
+
+        igniteEx = startGrid(0);
+
+        igniteEx.cluster().active(true);
+
+        long totalStartTimeInParallel = System.currentTimeMillis() - startTime;
+
+        for (int i = 0; i < CACHES_COUNT; i++)
+            igniteEx.cache(STATIC_CACHE_PREFIX + i).put(i, i);
+
+        stopAllGrids();
+
+        assertTrue("Consistently cache stat time : " + totalStartTimeConsistently +
+                "Parallelization cache stat time : " + totalStartTimeInParallel,
+            totalStartTimeConsistently > totalStartTimeInParallel);
+    }
+
+    /**
+     * Checking that start dynamic caches in parallel faster than consistenly.
+     *
+     * @throws Exception if fail.
+     */
+    public void testParallelizationAcceleratesStartOfCaches2() throws Exception {
+        //prepare dynamic caches.
+        isStaticCache = false;
+
+        IgniteEx igniteEx = startGrid(0);
+
+        igniteEx.cluster().active(true);
+
+        for (int i = 0; i < CACHES_COUNT; i++)
+            igniteEx.getOrCreateCache(DYNAMIC_CACHE_PREFIX + i);
+
+        stopAllGrids();
+
+        //start caches consistently.
+        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "false");
+
+        igniteEx = startGrid(0);
+        long startTime = System.currentTimeMillis();
+
+        igniteEx.cluster().active(true);
+
+        long totalStartTimeConsistently = System.currentTimeMillis() - startTime;
+
+        for (int i = 0; i < CACHES_COUNT; i++)
+            igniteEx.cache(DYNAMIC_CACHE_PREFIX + i);
+
+        stopAllGrids();
+
+        //start caches in parallel.
+        System.setProperty(IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL, "true");
+
+        startTime = System.currentTimeMillis();
+
+        igniteEx = startGrid(0);
+
+        igniteEx.cluster().active(true);
+
+        long totalStartTimeInParallel = System.currentTimeMillis() - startTime;
+
+        for (int i = 0; i < CACHES_COUNT; i++)
+            igniteEx.cache(DYNAMIC_CACHE_PREFIX + i).put(i, i);
+
+        stopAllGrids();
+
+        assertTrue("Consistently cache stat time : " + totalStartTimeConsistently +
+                "Parallelization cache stat time : " + totalStartTimeInParallel,
+            totalStartTimeConsistently > totalStartTimeInParallel);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
index 870ce67..66453b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
@@ -17,15 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed;
 
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
 import java.util.Collection;
 import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedDeque;
-import javax.cache.Cache;
-import javax.cache.configuration.Factory;
-import javax.cache.integration.CacheLoaderException;
-import javax.cache.integration.CacheWriterException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.store.CacheStore;
@@ -61,7 +61,7 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         CacheConfiguration cfg3 = cacheConfiguration("cacheC", new SecondStoreFactory());
         CacheConfiguration cfg4 = cacheConfiguration("cacheD", null);
 
-        cfg.setCacheConfiguration(cfg1, cfg2, cfg3, cfg4);
+        cfg.setCacheConfiguration(cfg4, cfg2, cfg3, cfg1);
 
         return cfg;
     }
@@ -92,6 +92,8 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
+        cleanPersistenceDir();
+
         startGrids(4);
     }
 
@@ -160,8 +162,8 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
             "delete cacheA",
             "write cacheB",
             "sessionEnd true"
-        ),
-        firstStoreEvts);
+            ),
+            firstStoreEvts);
 
         assertEquals(0, secondStoreEvts.size());
     }
@@ -209,16 +211,16 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
             "write cacheA",
             "delete cacheA",
             "sessionEnd true"
-        ),
-        firstStoreEvts);
+            ),
+            firstStoreEvts);
 
         assertEqualsCollections(F.asList(
             "writeAll cacheC 2",
             "deleteAll cacheC 2",
             "write cacheC",
             "sessionEnd true"
-        ),
-        secondStoreEvts);
+            ),
+            secondStoreEvts);
     }
 
     /**
@@ -264,8 +266,8 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
             "write cacheA",
             "delete cacheA",
             "sessionEnd true"
-        ),
-        firstStoreEvts);
+            ),
+            firstStoreEvts);
 
         assertEquals(0, secondStoreEvts.size());
     }
@@ -361,15 +363,10 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         private Ignite ignite;
 
         /** {@inheritDoc} */
-        @Override public CacheStore create() {
+        @Override public synchronized CacheStore create() {
             String igniteInstanceName = ignite.name();
 
-            CacheStore store = firstStores.get(igniteInstanceName);
-
-            if (store == null)
-                store = F.addIfAbsent(firstStores, igniteInstanceName, new TestStore());
-
-            return store;
+            return firstStores.computeIfAbsent(igniteInstanceName, (k) -> new TestStore());
         }
     }
 
@@ -384,12 +381,7 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         @Override public CacheStore create() {
             String igniteInstanceName = ignite.name();
 
-            CacheStore store = secondStores.get(igniteInstanceName);
-
-            if (store == null)
-                store = F.addIfAbsent(secondStores, igniteInstanceName, new TestStore());
-
-            return store;
+            return secondStores.computeIfAbsent(igniteInstanceName, (k) -> new TestStore());
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
index 61a076e..13a1044 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
@@ -44,6 +44,10 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterGroup;
@@ -877,6 +881,76 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testDoInParallel() throws Throwable {
+        CyclicBarrier barrier = new CyclicBarrier(3);
+
+        IgniteUtils.doInParallel(3,
+            Executors.newFixedThreadPool(3),
+            Arrays.asList(1, 2, 3),
+            i -> {
+                try {
+                    barrier.await(1, TimeUnit.SECONDS);
+                }
+                catch (Exception e) {
+                    throw new IgniteCheckedException(e);
+                }
+            }
+        );
+    }
+
+    /**
+     *
+     */
+    public void testDoInParallelBatch() {
+        CyclicBarrier barrier = new CyclicBarrier(3);
+
+        try {
+            IgniteUtils.doInParallel(2,
+                Executors.newFixedThreadPool(3),
+                Arrays.asList(1, 2, 3),
+                i -> {
+                    try {
+                        barrier.await(400, TimeUnit.MILLISECONDS);
+                    }
+                    catch (Exception e) {
+                        throw new IgniteCheckedException(e);
+                    }
+                }
+            );
+
+            fail("Should throw timeout exception");
+        }
+        catch (Exception e) {
+            assertTrue(e.getCause() instanceof TimeoutException);
+        }
+    }
+
+    /**
+     *
+     */
+    public void testDoInParallelException() {
+        String expectedException = "ExpectedException";
+
+        try {
+            IgniteUtils.doInParallel(3,
+                Executors.newFixedThreadPool(1),
+                Arrays.asList(1, 2, 3),
+                i -> {
+                    if (i == 1)
+                        throw new IgniteCheckedException(expectedException);
+                }
+            );
+
+            fail("Should throw ParallelExecutionException");
+        }
+        catch (IgniteCheckedException e) {
+            assertEquals(expectedException, e.getMessage());
+        }
+    }
+
+    /**
      * Test enum.
      */
     private enum TestEnum {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1f8f46f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
index d0734a8..0381a1f 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartition
 import org.apache.ignite.internal.processors.cache.distributed.CacheDataLossOnPartitionMoveTest;
 import org.apache.ignite.internal.processors.cache.distributed.CachePageWriteLockUnlockTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheRentingStateRepairTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheStartInParallelTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheStartWithLoadTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingPartitionCountersTest;
 import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingWithAsyncClearingTest;
@@ -99,6 +100,8 @@ public class IgniteCacheTestSuite7 extends TestSuite {
 
         suite.addTestSuite(CacheRentingStateRepairTest.class);
 
+        suite.addTestSuite(CacheStartInParallelTest.class);
+
         suite.addTestSuite(TransactionIntegrityWithPrimaryIndexCorruptionTest.class);
         suite.addTestSuite(CacheDataLossOnPartitionMoveTest.class);
 


[25/50] [abbrv] ignite git commit: IGNITE-9659 Fixed testNonCollocatedRetryMessage flaky test - Fixes #5005.

Posted by sb...@apache.org.
IGNITE-9659 Fixed testNonCollocatedRetryMessage flaky test - Fixes #5005.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/6f287428
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6f287428
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6f287428

Branch: refs/heads/ignite-gg-14206
Commit: 6f2874285a9eb1132d589223a5aa2c8fa8891fb8
Parents: f156631
Author: NSAmelchev <ns...@gmail.com>
Authored: Mon Oct 22 13:29:44 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 13:29:44 2018 +0300

----------------------------------------------------------------------
 .../NonCollocatedRetryMessageSelfTest.java      | 119 ++++++++++---------
 1 file changed, 62 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6f287428/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java
index c602225..e8f6624 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java
@@ -22,19 +22,18 @@ import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
-import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest;
+import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeRequest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT;
@@ -44,23 +43,31 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT;
  */
 public class NonCollocatedRetryMessageSelfTest extends GridCommonAbstractTest {
     /** */
-    private static final int NODES_COUNT = 3;
+    private static final int NODES_COUNT = 2;
 
     /** */
     private static final String ORG = "org";
 
     /** */
+    private static final int TEST_SQL_RETRY_TIMEOUT = 500;
+
+    /** */
+    private String sqlRetryTimeoutBackup;
+
+    /** */
     private IgniteCache<String, JoinSqlTestHelper.Person> personCache;
 
     /** */
     public void testNonCollocatedRetryMessage() {
-        SqlQuery<String, JoinSqlTestHelper.Person> qry = new SqlQuery<String, JoinSqlTestHelper.Person>(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0");
+        SqlQuery<String, JoinSqlTestHelper.Person> qry = new SqlQuery<String, JoinSqlTestHelper.Person>(
+            JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0");
 
         qry.setDistributedJoins(true);
 
         try {
-            List<Cache.Entry<String,JoinSqlTestHelper.Person>> prsns = personCache.query(qry).getAll();
-            fail("No CacheException emitted. Collection size="+prsns.size());
+            List<Cache.Entry<String, JoinSqlTestHelper.Person>> prsns = personCache.query(qry).getAll();
+
+            fail("No CacheException emitted. Collection size=" + prsns.size());
         }
         catch (CacheException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("Failed to execute non-collocated query"));
@@ -68,66 +75,35 @@ public class NonCollocatedRetryMessageSelfTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setCommunicationSpi(new TcpCommunicationSpi(){
-            volatile long reqId = -1;
-            /** {@inheritDoc} */
-            @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) {
-                assert msg != null;
-
-                if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){
-                    GridIoMessage gridMsg = (GridIoMessage)msg;
-
-                    if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){
-                        GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message());
-
-                        if (reqId < 0) {
-                            reqId = req.requestId();
-
-                            String shutName = getTestIgniteInstanceName(1);
-
-                            stopGrid(shutName, true, false);
-                        }
-                        else if( reqId != req.requestId() ){
-                            try {
-                                U.sleep(IgniteSystemProperties.getLong(IGNITE_SQL_RETRY_TIMEOUT, GridReduceQueryExecutor.DFLT_RETRY_TIMEOUT));
-                            }
-                            catch (IgniteInterruptedCheckedException e) {
-                                // no-op
-                            }
-                        }
-                    }
-                }
-                super.sendMessage(node, msg, ackC);
-            }
-        });
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setDiscoverySpi(new TcpDiscoverySpi(){
-            public long getNodesJoined() {
-                return stats.joinedNodesCount();
-            }
-        });
+        cfg.setCommunicationSpi(new TestTcpCommunication());
 
         return cfg;
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000");
+        sqlRetryTimeoutBackup = System.getProperty(IGNITE_SQL_RETRY_TIMEOUT);
+
+        System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, String.valueOf(TEST_SQL_RETRY_TIMEOUT));
 
         startGridsMultiThreaded(NODES_COUNT, false);
 
-        personCache = ignite(0).getOrCreateCache(new CacheConfiguration<String, JoinSqlTestHelper.Person>("pers")
-            .setBackups(1)
-            .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class)
-        );
+        CacheConfiguration<String, JoinSqlTestHelper.Person> ccfg1 = new CacheConfiguration<>("pers");
+
+        ccfg1.setBackups(1);
+        ccfg1.setIndexedTypes(String.class, JoinSqlTestHelper.Person.class);
 
-        final IgniteCache<String, JoinSqlTestHelper.Organization> orgCache = ignite(0).getOrCreateCache(new CacheConfiguration<String, JoinSqlTestHelper.Organization>(ORG)
-            .setBackups(1)
-            .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class)
-        );
+        personCache = ignite(0).getOrCreateCache(ccfg1);
+
+        CacheConfiguration<String, JoinSqlTestHelper.Organization> ccfg2 = new CacheConfiguration<>(ORG);
+
+        ccfg2.setBackups(1);
+        ccfg2.setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class);
+
+        IgniteCache<String, JoinSqlTestHelper.Organization> orgCache = ignite(0).getOrCreateCache(ccfg2);
 
         awaitPartitionMapExchange();
 
@@ -137,10 +113,39 @@ public class NonCollocatedRetryMessageSelfTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
+    @Override protected void afterTest() {
+        if (sqlRetryTimeoutBackup != null)
+            System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, sqlRetryTimeoutBackup);
+
         stopAllGrids();
     }
 
+    /**
+     * TcpCommunicationSpi with additional features needed for tests.
+     */
+    private class TestTcpCommunication extends TcpCommunicationSpi {
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) {
+            assert msg != null;
+
+            if (igniteInstanceName.equals(getTestIgniteInstanceName(1)) &&
+                GridIoMessage.class.isAssignableFrom(msg.getClass())) {
+                GridIoMessage gridMsg = (GridIoMessage)msg;
+
+                if (GridH2IndexRangeRequest.class.isAssignableFrom(gridMsg.message().getClass())) {
+                    try {
+                        U.sleep(TEST_SQL_RETRY_TIMEOUT);
+                    }
+                    catch (IgniteInterruptedCheckedException e) {
+                        fail("Test was interrupted.");
+                    }
 
+                    throw new IgniteSpiException("Test exception.");
+                }
+            }
+
+            super.sendMessage(node, msg, ackC);
+        }
+    }
 }
 


[19/50] [abbrv] ignite git commit: IGNITE-9776 Fixed infinite block in FsyncModeFileWriteAheadLogManager log() call - Fixes #4953.

Posted by sb...@apache.org.
IGNITE-9776 Fixed infinite block in FsyncModeFileWriteAheadLogManager log() call - Fixes #4953.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7deec60a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7deec60a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7deec60a

Branch: refs/heads/ignite-gg-14206
Commit: 7deec60a8727d86fe153962979c3c9f4d67d8e09
Parents: 5427c09
Author: Alexey Stelmak <sp...@gmail.com>
Authored: Fri Oct 19 12:11:32 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 19 12:15:48 2018 +0300

----------------------------------------------------------------------
 .../wal/FsyncModeFileWriteAheadLogManager.java  | 27 ++++--
 .../wal/FsyncWalRolloverDoesNotBlockTest.java   | 89 ++++++++++++++++++++
 .../db/wal/WalRolloverTypesTest.java            |  8 --
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |  3 +
 .../testsuites/IgnitePdsNativeIoTestSuite2.java |  1 +
 5 files changed, 112 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7deec60a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
index 917640d..0ede897 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
@@ -419,9 +419,9 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
 
             lastTruncatedArchiveIdx = tup == null ? -1 : tup.get1() - 1;
 
-            archiver = new FileArchiver(tup == null ? -1 : tup.get2(), log);
+            archiver = isArchiverEnabled() ? new FileArchiver(tup == null ? -1 : tup.get2(), log) : null;
 
-            if (dsCfg.isWalCompactionEnabled()) {
+            if (archiver != null && dsCfg.isWalCompactionEnabled()) {
                 compressor = new FileCompressor();
 
                 if (decompressor == null) {  // Preventing of two file-decompressor thread instantiations.
@@ -619,7 +619,7 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
     public Collection<File> getAndReserveWalFiles(FileWALPointer low, FileWALPointer high) throws IgniteCheckedException {
         final long awaitIdx = high.index() - 1;
 
-        while (archiver.lastArchivedAbsoluteIndex() < awaitIdx)
+        while (archiver != null && archiver.lastArchivedAbsoluteIndex() < awaitIdx)
             LockSupport.parkNanos(Thread.currentThread(), 1_000_000);
 
         if (!reserve(low))
@@ -918,6 +918,9 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
 
     /** {@inheritDoc} */
     @Override public int walArchiveSegments() {
+        if (archiver == null)
+            return 0;
+
         long lastTruncated = lastTruncatedArchiveIdx;
 
         long lastArchived = archiver.lastArchivedAbsoluteIndex();
@@ -966,7 +969,7 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
 
     /** {@inheritDoc} */
     @Override public long lastArchivedSegment() {
-        return archiver.lastArchivedAbsoluteIndex();
+        return archiver != null ? archiver.lastArchivedAbsoluteIndex() : -1L;
     }
 
     /** {@inheritDoc} */
@@ -1235,7 +1238,8 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
                 if (lastReadPtr == null)
                     hnd.writeSerializerVersion();
 
-                archiver.currentWalIndex(absIdx);
+                if (archiver != null)
+                    archiver.currentWalIndex(absIdx);
 
                 return hnd;
             }
@@ -1411,8 +1415,13 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
      * @throws IgniteInterruptedCheckedException If interrupted.
      */
     private File pollNextFile(long curIdx) throws StorageException, IgniteInterruptedCheckedException {
+        FileArchiver archiver0 = archiver;
+
+        if (archiver0 == null)
+            return new File(walWorkDir, FileDescriptor.fileName(curIdx + 1));
+
         // Signal to archiver that we are done with the segment and it can be archived.
-        long absNextIdx = archiver.nextAbsoluteSegmentIndex(curIdx);
+        long absNextIdx = archiver0.nextAbsoluteSegmentIndex(curIdx);
 
         long segmentIdx = absNextIdx % dsCfg.getWalSegments();
 
@@ -1724,8 +1733,10 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
 
                     int segments = dsCfg.getWalSegments();
 
-                    while ((curAbsWalIdx - lastAbsArchivedIdx > segments && cleanException == null))
-                        wait();
+                    if (isArchiverEnabled()) {
+                        while ((curAbsWalIdx - lastAbsArchivedIdx > segments && cleanException == null))
+                            wait();
+                    }
 
                     if (cleanException != null)
                         throw cleanException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7deec60a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/FsyncWalRolloverDoesNotBlockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/FsyncWalRolloverDoesNotBlockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/FsyncWalRolloverDoesNotBlockTest.java
new file mode 100644
index 0000000..aa2e90c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/FsyncWalRolloverDoesNotBlockTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.persistence.db.wal;
+
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_PATH;
+import static org.apache.ignite.configuration.WALMode.FSYNC;
+
+/** */
+public class FsyncWalRolloverDoesNotBlockTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static class RolloverRecord extends CheckpointRecord {
+        /** */
+        private RolloverRecord() {
+            super(null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))
+            .setWalMode(FSYNC)
+            .setWalArchivePath(DFLT_WAL_PATH));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    public void test() throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().active(true);
+
+        ig.context().cache().context().database().checkpointReadLock();
+
+        try {
+            ig.context().cache().context().wal().log(new RolloverRecord());
+        }
+        finally {
+            ig.context().cache().context().database().checkpointReadUnlock();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7deec60a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRolloverTypesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRolloverTypesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRolloverTypesTest.java
index 122ecb6..db56cd2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRolloverTypesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRolloverTypesTest.java
@@ -117,8 +117,6 @@ public class WalRolloverTypesTest extends GridCommonAbstractTest {
 
     /** */
     public void testCurrentSegmentTypeLogFsyncModeArchiveOff() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9776");
-
         checkCurrentSegmentType(FSYNC, true);
     }
 
@@ -139,8 +137,6 @@ public class WalRolloverTypesTest extends GridCommonAbstractTest {
 
     /** */
     public void testNextSegmentTypeFsyncModeArchiveOff() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9776");
-
         checkNextSegmentType(FSYNC, true);
     }
 
@@ -207,8 +203,6 @@ public class WalRolloverTypesTest extends GridCommonAbstractTest {
 
     /** */
     public void testNextSegmentTypeWithCacheActivityFsyncModeArchiveOff() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9776");
-
         checkNextSegmentTypeWithCacheActivity(FSYNC, true);
     }
 
@@ -295,8 +289,6 @@ public class WalRolloverTypesTest extends GridCommonAbstractTest {
 
     /** */
     public void testCurrentSegmentTypeWithCacheActivityFsyncModeArchiveOff() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9776");
-
         checkCurrentSegmentTypeWithCacheActivity(FSYNC, true);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7deec60a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index ede5cab..75a8af7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWhole
 import org.apache.ignite.internal.processors.cache.persistence.db.SlowHistoricalRebalanceSmallHistoryTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.checkpoint.IgniteCheckpointDirtyPagesForLowLoadTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.filename.IgniteUidAsConsistentIdMigrationTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.wal.FsyncWalRolloverDoesNotBlockTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteNodeStoppedDuringDisableWALTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWALTailIsReachedDuringIterationOverArchiveTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushBackgroundSelfTest;
@@ -191,5 +192,7 @@ public class IgnitePdsTestSuite2 extends TestSuite {
         suite.addTestSuite(IgniteWALTailIsReachedDuringIterationOverArchiveTest.class);
 
         suite.addTestSuite(WalRolloverTypesTest.class);
+
+        suite.addTestSuite(FsyncWalRolloverDoesNotBlockTest.class);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7deec60a/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java b/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
index 2ed7450..7aebe95 100644
--- a/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
+++ b/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteNativeIoLocalWalModeChangeDuringRebalancingSelfTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteNativeIoPdsRecoveryAfterFileCorruptionTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.wal.FsyncWalRolloverDoesNotBlockTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteNativeIoWalFlushFsyncSelfTest;
 
 /**


[50/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-gg-14206

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-gg-14206


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1e91d477
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1e91d477
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1e91d477

Branch: refs/heads/ignite-gg-14206
Commit: 1e91d477f8db279232a43ee66ffc17c23b0193d2
Parents: 1a7a9c7 86f5437
Author: sboikov <sb...@apache.org>
Authored: Tue Oct 23 18:00:11 2018 +0300
Committer: sboikov <sb...@apache.org>
Committed: Tue Oct 23 18:00:11 2018 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   14 +-
 README.txt                                      |    4 +-
 RELEASE_NOTES.txt                               |   76 +-
 assembly/release-apache-ignite-base.xml         |   71 +
 bin/control.bat                                 |    6 +-
 bin/ignite-tf.sh                                |  183 ++
 bin/ignite.bat                                  |   54 +-
 bin/ignite.sh                                   |   37 +-
 bin/ignitevisorcmd.bat                          |    6 +-
 bin/include/functions.sh                        |    9 +-
 config/apache-ignite-lgpl/default-config.xml    |   29 +
 config/apache-ignite/default-config.xml         |   29 +
 config/fabric-lgpl/default-config.xml           |   29 -
 config/fabric/default-config.xml                |   29 -
 doap_Ignite.rdf                                 |    4 +-
 .../encryption/example-encrypted-store.xml      |   69 +
 examples/config/encryption/example_keystore.jks |  Bin 0 -> 347 bytes
 .../encryption/EncryptedCacheExample.java       |  106 +
 .../examples/EncryptedCacheExampleSelfTest.java |   36 +
 .../testsuites/IgniteExamplesSelfTestSuite.java |    4 +
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |    8 +-
 modules/benchmarks/pom.xml                      |   12 +-
 .../benchmarks/jmh/algo/BenchmarkCRC.java       |   95 +
 .../JmhKeystoreEncryptionSpiBenchmark.java      |  117 +
 ...ridDhtPartitionsStateValidatorBenchmark.java |  185 ++
 .../jmh/notify/JmhWaitStategyBenchmark.java     |    4 +-
 .../jdbc2/JdbcDynamicIndexAbstractSelfTest.java |    8 +-
 .../jdbc/JdbcVersionMismatchSelfTest.java       |  171 ++
 .../suite/IgniteJdbcDriverMvccTestSuite.java    |    4 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    2 +
 .../JdbcThinConnectionMvccEnabledSelfTest.java  |    8 +-
 .../JdbcThinMetadataPrimaryKeysSelfTest.java    |  152 ++
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |    2 +-
 ...ThinTransactionsAbstractComplexSelfTest.java |    8 +-
 .../jdbc/thin/JdbcThinTransactionsSelfTest.java |    4 +-
 ...ThinTransactionsWithMvccEnabledSelfTest.java |   15 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |    2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |    7 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    3 -
 .../apache/ignite/IgniteSystemProperties.java   |   35 +-
 .../configuration/CacheConfiguration.java       |   37 +-
 .../configuration/DataStorageConfiguration.java |   27 +
 .../configuration/IgniteConfiguration.java      |   56 +
 .../apache/ignite/internal/GridComponent.java   |    9 +-
 .../ignite/internal/GridKernalContext.java      |    8 +
 .../ignite/internal/GridKernalContextImpl.java  |   12 +
 .../org/apache/ignite/internal/GridTopic.java   |    5 +-
 .../internal/IgniteDiagnosticMessage.java       |    8 +-
 .../IgniteDiagnosticPrepareContext.java         |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  107 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../org/apache/ignite/internal/IgnitionEx.java  |   10 +-
 .../internal/binary/BinaryFieldAccessor.java    |    5 +-
 .../internal/binary/BinaryReaderExImpl.java     |    1 -
 .../internal/binary/BinaryReaderHandles.java    |    2 +-
 .../internal/binary/BinarySchemaRegistry.java   |  114 +-
 .../ignite/internal/binary/BinaryUtils.java     |   28 +-
 .../commandline/cache/CacheArguments.java       |    2 +-
 .../internal/dto/IgniteDataTransferObject.java  |  130 +
 .../dto/IgniteDataTransferObjectInput.java      |  156 ++
 .../dto/IgniteDataTransferObjectOutput.java     |  141 +
 .../internal/jdbc/thin/JdbcThinConnection.java  |    4 +-
 .../internal/jdbc/thin/JdbcThinSSLUtil.java     |    2 +-
 .../communication/GridIoMessageFactory.java     |   40 +-
 .../discovery/GridDiscoveryManager.java         |   57 +-
 .../GenerateEncryptionKeyRequest.java           |  142 +
 .../GenerateEncryptionKeyResponse.java          |  148 ++
 .../encryption/GridEncryptionManager.java       |  864 ++++++
 .../ignite/internal/pagemem/PageMemory.java     |    6 +
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |    5 +
 .../internal/pagemem/store/PageStore.java       |   27 +
 .../pagemem/wal/IgniteWriteAheadLogManager.java |   19 +-
 .../pagemem/wal/record/EncryptedRecord.java     |   60 +
 .../pagemem/wal/record/PageSnapshot.java        |   19 +-
 .../pagemem/wal/record/RolloverType.java        |   38 +
 .../pagemem/wal/record/SnapshotRecord.java      |    5 -
 .../internal/pagemem/wal/record/WALRecord.java  |   15 +-
 .../delta/DataPageInsertFragmentRecord.java     |    2 +-
 .../wal/record/delta/DataPageInsertRecord.java  |    2 +-
 .../delta/DataPageMvccMarkUpdatedRecord.java    |    2 +-
 .../DataPageMvccUpdateNewTxStateHintRecord.java |    2 +-
 .../DataPageMvccUpdateTxStateHintRecord.java    |    2 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |    2 +-
 .../wal/record/delta/DataPageUpdateRecord.java  |    2 +-
 .../wal/record/delta/InitNewPageRecord.java     |    2 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |    2 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |    2 +-
 .../wal/record/delta/MetaPageInitRecord.java    |    2 +-
 .../delta/MetaPageInitRootInlineRecord.java     |    2 +-
 .../record/delta/MetaPageInitRootRecord.java    |    2 +-
 .../wal/record/delta/NewRootInitRecord.java     |    3 +-
 .../record/delta/PagesListAddPageRecord.java    |    2 +-
 .../delta/PagesListInitNewPageRecord.java       |    4 +-
 .../record/delta/TrackingPageDeltaRecord.java   |    2 +-
 .../IgniteAuthenticationProcessor.java          |   89 +-
 .../cache/CacheAffinitySharedManager.java       |  341 ++-
 .../cache/CacheConflictResolutionManager.java   |    2 +-
 .../CacheDefaultBinaryAffinityKeyMapper.java    |    2 +-
 .../processors/cache/CacheGroupContext.java     |    8 +
 .../processors/cache/CacheGroupDescriptor.java  |   19 +
 .../processors/cache/CacheInvokeEntry.java      |   45 +-
 .../processors/cache/ClusterCachesInfo.java     |   34 +-
 .../cache/DynamicCacheChangeRequest.java        |   17 +
 .../processors/cache/GridCacheAdapter.java      |   11 +-
 .../processors/cache/GridCacheAttributes.java   |    9 +-
 .../processors/cache/GridCacheEntryEx.java      |   35 +-
 .../processors/cache/GridCacheIoManager.java    |   22 +-
 .../processors/cache/GridCacheMapEntry.java     |  329 ++-
 .../cache/GridCacheMvccEntryInfo.java           |    5 -
 .../processors/cache/GridCacheMvccManager.java  |    2 +
 .../GridCachePartitionExchangeManager.java      |  227 +-
 .../processors/cache/GridCacheProcessor.java    |  937 +++++--
 .../processors/cache/GridCacheProxyImpl.java    |    7 +-
 .../cache/GridCacheSharedContext.java           |   52 +-
 .../cache/GridCacheUpdateTxResult.java          |   78 +-
 .../processors/cache/GridCacheUtils.java        |   25 +-
 .../cache/IgniteCacheOffheapManager.java        |   37 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  130 +-
 .../processors/cache/IgniteCacheProxyImpl.java  |   95 +-
 .../cache/PartitionUpdateCounter.java           |   30 +-
 .../processors/cache/QueryCursorImpl.java       |    2 +-
 .../processors/cache/StartCacheInfo.java        |  113 +
 .../cache/binary/BinaryMetadataTransport.java   |   93 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  219 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |   15 +-
 .../distributed/GridDistributedCacheEntry.java  |    2 +-
 .../distributed/GridDistributedTxMapping.java   |   32 +
 .../GridDistributedTxRemoteAdapter.java         |  607 ++---
 .../dht/GridDhtTopologyFutureAdapter.java       |    3 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |    3 +-
 .../dht/GridDhtTxAbstractEnlistFuture.java      |   59 +-
 .../distributed/dht/GridDhtTxEnlistFuture.java  |   22 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   36 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   10 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   41 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   66 +-
 .../dht/GridDhtTxQueryEnlistRequest.java        |    8 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |  133 -
 .../cache/distributed/dht/GridInvokeValue.java  |  186 ++
 .../dht/GridPartitionedGetFuture.java           |    7 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    7 +-
 .../dht/NearTxQueryEnlistResultHandler.java     |    4 +-
 .../GridNearAtomicSingleUpdateFuture.java       |    2 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |    2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    5 +-
 .../preloader/CacheGroupAffinityMessage.java    |   12 +-
 .../CachePartitionFullCountersMap.java          |   21 -
 .../dht/preloader/GridDhtPartitionDemander.java |   20 +-
 .../dht/preloader/GridDhtPartitionSupplier.java |   13 +-
 .../GridDhtPartitionsExchangeFuture.java        |  570 ++--
 .../GridDhtPartitionsSingleMessage.java         |   29 +
 .../topology/GridClientPartitionTopology.java   |    3 +-
 .../dht/topology/GridDhtLocalPartition.java     |    7 +
 .../dht/topology/GridDhtPartitionTopology.java  |    5 +-
 .../topology/GridDhtPartitionTopologyImpl.java  |   16 +-
 .../GridDhtPartitionsStateValidator.java        |  113 +-
 .../dht/topology/PartitionsEvictManager.java    |   65 +-
 .../distributed/near/GridNearGetFuture.java     |   27 +-
 ...arOptimisticSerializableTxPrepareFuture.java |    2 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |    2 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   42 +-
 .../near/GridNearTxAbstractEnlistFuture.java    |   28 +-
 .../near/GridNearTxEnlistFuture.java            |   30 +-
 .../near/GridNearTxEnlistRequest.java           |   35 +-
 .../near/GridNearTxEnlistResponse.java          |    7 +
 .../near/GridNearTxFinishFuture.java            |   38 +-
 .../cache/distributed/near/GridNearTxLocal.java |   90 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |   20 +-
 .../near/GridNearTxQueryEnlistFuture.java       |   18 +-
 .../GridNearTxQueryResultsEnlistFuture.java     |   11 +-
 .../processors/cache/dr/GridCacheDrManager.java |   34 +-
 .../cache/dr/GridOsCacheDrManager.java          |   15 +-
 .../cache/local/GridLocalLockFuture.java        |    2 +-
 .../cache/mvcc/MvccCachingManager.java          |  341 +++
 .../processors/cache/mvcc/MvccProcessor.java    |    7 -
 .../cache/mvcc/MvccProcessorImpl.java           |  453 ++--
 .../cache/mvcc/MvccQueryTrackerImpl.java        |   16 +-
 .../processors/cache/mvcc/MvccTxEntry.java      |  203 ++
 .../cache/mvcc/MvccUpdateVersionAware.java      |    5 -
 .../processors/cache/mvcc/MvccUtils.java        |   20 +-
 .../mvcc/msg/MvccRecoveryFinishedMessage.java   |  116 +
 .../cache/mvcc/msg/MvccSnapshotResponse.java    |    4 +
 .../PartitionCountersNeighborcastRequest.java   |  145 +
 .../PartitionCountersNeighborcastResponse.java  |  114 +
 .../cache/persistence/CacheDataRowAdapter.java  |    7 +-
 .../cache/persistence/DataStructure.java        |    6 +-
 .../cache/persistence/DbCheckpointListener.java |    2 +-
 .../GridCacheDatabaseSharedManager.java         |  265 +-
 .../persistence/GridCacheOffheapManager.java    |   88 +-
 .../IgniteCacheDatabaseSharedManager.java       |   20 +-
 .../cache/persistence/file/EncryptedFileIO.java |  372 +++
 .../file/EncryptedFileIOFactory.java            |  100 +
 .../cache/persistence/file/FilePageStore.java   |   35 +-
 .../persistence/file/FilePageStoreFactory.java  |    3 +-
 .../persistence/file/FilePageStoreManager.java  |   76 +-
 .../file/FileVersionCheckingFactory.java        |   25 +-
 .../cache/persistence/freelist/PagesList.java   |    4 +-
 .../persistence/metastorage/MetaStorage.java    |    2 +
 .../persistence/pagemem/PageMemoryImpl.java     |   46 +-
 .../partstate/PartitionAllocationMap.java       |   55 +-
 .../cache/persistence/tree/io/BPlusIO.java      |    4 +-
 .../cache/persistence/tree/io/DataPageIO.java   |   15 +-
 .../cache/persistence/tree/io/PageIO.java       |    5 +-
 .../tree/io/PagePartitionCountersIO.java        |    2 +-
 .../persistence/tree/util/PageHandler.java      |   50 +-
 .../wal/FileWriteAheadLogManager.java           |  117 +-
 .../wal/FsyncModeFileWriteAheadLogManager.java  |  114 +-
 .../wal/SegmentedRingByteBuffer.java            |    4 +-
 .../cache/persistence/wal/crc/FastCrc.java      |  101 +
 .../persistence/wal/crc/PureJavaCrc32.java      |    4 +-
 .../cache/persistence/wal/io/FileInput.java     |   10 +-
 .../persistence/wal/io/SimpleFileInput.java     |    3 +-
 .../wal/reader/IgniteWalIteratorFactory.java    |    2 +-
 .../wal/reader/StandaloneGridKernalContext.java |    6 +
 .../reader/StandaloneWalRecordsIterator.java    |    3 +
 .../wal/serializer/RecordDataV1Serializer.java  |  355 ++-
 .../wal/serializer/RecordDataV2Serializer.java  |   60 +-
 .../serializer/RecordSerializerFactoryImpl.java |    5 +-
 .../wal/serializer/RecordV1Serializer.java      |   13 +-
 .../wal/serializer/RecordV2Serializer.java      |    2 +-
 .../cache/query/GridCacheQueryAdapter.java      |    2 +-
 .../cache/query/IgniteQueryErrorCode.java       |   62 +-
 .../continuous/CacheContinuousQueryHandler.java |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   20 +-
 .../CacheContinuousQueryPartitionRecovery.java  |    2 +-
 .../cache/transactions/IgniteTxAdapter.java     |  112 +-
 .../cache/transactions/IgniteTxHandler.java     |  415 ++-
 .../IgniteTxImplicitSingleStateImpl.java        |    2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  542 ++--
 .../cache/transactions/IgniteTxManager.java     |  144 +-
 .../IgniteTxRemoteSingleStateImpl.java          |    2 +-
 .../transactions/IgniteTxRemoteStateImpl.java   |    2 +-
 .../cache/transactions/IgniteTxState.java       |    3 +-
 .../cache/transactions/IgniteTxStateImpl.java   |    2 +-
 .../PartitionCountersNeighborcastFuture.java    |  211 ++
 .../cache/transactions/TxCounters.java          |    3 +-
 .../cache/tree/mvcc/data/MvccDataRow.java       |   64 +-
 .../cache/tree/mvcc/data/MvccUpdateDataRow.java |  127 +-
 .../cache/tree/mvcc/data/MvccUpdateResult.java  |   17 +-
 .../cache/tree/mvcc/data/ResultType.java        |    4 +-
 .../cache/version/GridCacheVersionManager.java  |   31 +-
 .../processors/cluster/BaselineTopology.java    |   14 +-
 .../cluster/ChangeGlobalStateMessage.java       |    6 +-
 .../cluster/DiscoveryDataClusterState.java      |   17 +
 .../cluster/GridClusterStateProcessor.java      |   52 +-
 .../processors/cluster/GridUpdateNotifier.java  |    4 +-
 .../cluster/IGridClusterStateProcessor.java     |    6 +
 .../continuous/GridContinuousProcessor.java     |   18 +-
 .../datastreamer/DataStreamerImpl.java          |   36 +-
 .../processors/failure/FailureProcessor.java    |   20 +-
 .../processors/hadoop/HadoopClassLoader.java    |   37 +-
 .../processors/hadoop/HadoopHelper.java         |    5 +
 .../internal/processors/hadoop/HadoopJobEx.java |   20 +-
 .../processors/hadoop/HadoopNoopHelper.java     |    5 +
 .../internal/processors/igfs/IgfsImpl.java      |    3 +
 .../processors/igfs/IgfsMetaManager.java        |    6 +-
 .../internal/processors/odbc/SqlStateCode.java  |   31 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |    5 +-
 .../odbc/jdbc/JdbcRequestHandlerWorker.java     |    2 +-
 .../odbc/odbc/OdbcRequestHandler.java           |   33 +-
 .../odbc/odbc/OdbcRequestHandlerWorker.java     |    2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |    2 +-
 .../utils/PlatformConfigurationUtils.java       |  101 +-
 .../processors/query/EnlistOperation.java       |   11 +-
 .../processors/query/GridQueryProcessor.java    |    8 +-
 .../internal/processors/query/NestedTxMode.java |    2 +-
 .../internal/processors/query/QueryUtils.java   |    2 +-
 .../protocols/tcp/redis/GridRedisMessage.java   |    2 +-
 .../processors/security/SecurityUtils.java      |    2 +-
 .../processors/task/GridTaskProcessor.java      |   25 +-
 .../sql/command/SqlCreateIndexCommand.java      |    2 +-
 .../sql/command/SqlSetStreamingCommand.java     |    2 +-
 .../ignite/internal/util/GridLongList.java      |    6 +
 .../ignite/internal/util/IgniteUtils.java       |  309 ++-
 .../internal/util/InitializationProtector.java  |   79 +
 .../ignite/internal/util/StripedExecutor.java   |    2 +-
 .../ignite/internal/util/lang/GridFunc.java     |    8 +-
 .../util/lang/IgniteThrowableConsumer.java      |   37 +
 .../util/lang/IgniteThrowableRunner.java        |   30 +
 .../ignite/internal/util/nio/GridNioServer.java |    6 +
 .../ignite/internal/util/worker/GridWorker.java |    4 +-
 .../internal/visor/VisorDataTransferObject.java |    8 +-
 .../visor/VisorDataTransferObjectInput.java     |    2 +
 .../visor/VisorDataTransferObjectOutput.java    |    2 +
 .../visor/debug/VisorThreadMonitorInfo.java     |    5 -
 .../visor/event/VisorGridDeploymentEvent.java   |    5 -
 .../visor/event/VisorGridDiscoveryEvent.java    |    5 -
 .../internal/visor/event/VisorGridJobEvent.java |    5 -
 .../visor/event/VisorGridTaskEvent.java         |    5 -
 .../visor/node/VisorGridConfiguration.java      |   22 +-
 .../visor/node/VisorMvccConfiguration.java      |   94 +
 .../ignite/internal/visor/tx/VisorTxInfo.java   |    4 +-
 .../internal/visor/tx/VisorTxTaskArg.java       |   18 +-
 .../visor/verify/VisorViewCacheTaskArg.java     |    2 +-
 .../worker/FailureHandlingMxBeanImpl.java       |   73 +
 .../worker/WorkersControlMXBeanImpl.java        |   10 -
 .../ignite/internal/worker/WorkersRegistry.java |   47 +-
 .../ignite/mxbean/FailureHandlingMxBean.java    |   47 +
 .../ignite/mxbean/WorkersControlMXBean.java     |    7 -
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    1 -
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   13 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    5 +-
 .../ignite/spi/encryption/EncryptionSpi.java    |  113 +
 .../keystore/KeystoreEncryptionKey.java         |   84 +
 .../keystore/KeystoreEncryptionSpi.java         |  501 ++++
 .../spi/encryption/keystore/package-info.java   |   22 +
 .../spi/encryption/noop/NoopEncryptionSpi.java  |  101 +
 .../spi/encryption/noop/package-info.java       |   22 +
 .../ignite/spi/encryption/package-info.java     |   22 +
 .../IgniteTaskTrackingThreadPoolExecutor.java   |  180 --
 .../apache/ignite/transactions/Transaction.java |    2 +-
 .../core/src/main/java/org/mindrot/BCrypt.java  |    6 +-
 ...reSessionListenerWriteBehindEnabledTest.java |    4 +-
 .../ignite/client/ClientConfigurationTest.java  |   42 +
 .../FailureHandlingConfigurationTest.java       |  262 ++
 .../failure/SystemWorkersTerminationTest.java   |   72 +-
 .../ignite/failure/TestFailureHandler.java      |   10 +-
 .../IgniteClientReconnectApiExceptionTest.java  |    2 +-
 .../encryption/AbstractEncryptionTest.java      |  245 ++
 .../encryption/EncryptedCacheBigEntryTest.java  |  114 +
 .../encryption/EncryptedCacheCreateTest.java    |  164 ++
 .../encryption/EncryptedCacheDestroyTest.java   |  127 +
 .../EncryptedCacheGroupCreateTest.java          |  116 +
 .../encryption/EncryptedCacheNodeJoinTest.java  |  237 ++
 .../EncryptedCachePreconfiguredRestartTest.java |   87 +
 .../encryption/EncryptedCacheRestartTest.java   |   64 +
 .../IgniteTopologyPrintFormatSelfTest.java      |   19 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |   18 +-
 .../GridAffinityProcessorMemoryLeakTest.java    |    2 +-
 .../AuthenticationProcessorNodeRestartTest.java |  340 ++-
 ...ataRegistrationInsideEntryProcessorTest.java |   49 +-
 .../cache/CacheGetEntryAbstractTest.java        |    4 +-
 .../cache/ClusterReadOnlyModeAbstractTest.java  |  114 +
 .../cache/ClusterReadOnlyModeTest.java          |  134 +
 .../GridCacheAbstractFailoverSelfTest.java      |   20 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   28 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |   46 +-
 .../cache/GridCacheAbstractSelfTest.java        |   11 +-
 ...GridCacheConcurrentGetCacheOnClientTest.java |    2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   17 +-
 .../IgniteCacheConfigVariationsFullApiTest.java |   62 +-
 .../processors/cache/MapCacheStoreStrategy.java |    8 +-
 .../binary/BinaryMetadataUpdatesFlowTest.java   |    6 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |    2 +-
 .../distributed/CacheBlockOnGetAllTest.java     |  196 ++
 .../CacheBlockOnReadAbstractTest.java           | 1277 +++++++++
 .../cache/distributed/CacheBlockOnScanTest.java |   73 +
 .../distributed/CacheBlockOnSingleGetTest.java  |  190 ++
 .../distributed/CacheExchangeMergeTest.java     |    4 +-
 .../CachePageWriteLockUnlockTest.java           |    2 +
 ...CacheResultIsNotNullOnPartitionLossTest.java |  213 ++
 .../IgniteCacheTxIteratorSelfTest.java          |    2 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |   44 +-
 .../IgniteOptimisticTxSuspendResumeTest.java    |    6 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |    2 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   69 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |    2 +-
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |    2 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |  133 +-
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |    4 +-
 .../near/GridCacheNearClientHitTest.java        |    4 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   20 +-
 .../GridCacheRebalancingOrderingTest.java       |    2 +-
 ...licatedTxOriginatingNodeFailureSelfTest.java |    2 +-
 .../mvcc/CacheMvccAbstractFeatureTest.java      |    2 +-
 .../cache/mvcc/CacheMvccAbstractTest.java       |  103 +-
 .../CacheMvccConfigurationValidationTest.java   |  287 +-
 .../mvcc/CacheMvccOperationChecksTest.java      |    4 +-
 .../CacheMvccRemoteTxOnNearNodeStartTest.java   |   90 +
 .../cache/mvcc/CacheMvccTransactionsTest.java   |   93 +-
 ...gnitePdsRecoveryAfterFileCorruptionTest.java |    8 +-
 .../IgnitePersistentStoreCacheGroupsTest.java   |   51 +-
 ...ckpointSimulationWithRealCpDisabledTest.java |   21 +-
 .../db/file/IgnitePdsPageReplacementTest.java   |    2 +-
 .../wal/FsyncWalRolloverDoesNotBlockTest.java   |   89 +
 .../wal/IgniteWalIteratorSwitchSegmentTest.java |    3 +-
 .../persistence/db/wal/WalCompactionTest.java   |   12 +-
 .../db/wal/WalDeletionArchiveAbstractTest.java  |    2 +-
 .../db/wal/WalRolloverRecordLoggingTest.java    |    8 +-
 .../db/wal/WalRolloverTypesTest.java            |  358 +++
 .../db/wal/crc/IgniteDataIntegrityTests.java    |    6 +-
 .../wal/crc/IgnitePureJavaCrcCompatibility.java |   55 +
 .../metastorage/IgniteMetaStorageBasicTest.java |   45 +
 .../pagemem/BPlusTreePageMemoryImplTest.java    |   23 +-
 .../BPlusTreeReuseListPageMemoryImplTest.java   |   19 +-
 ...gnitePageMemReplaceDelayedWriteUnitTest.java |    7 +-
 .../pagemem/IndexStoragePageMemoryImplTest.java |   23 +-
 .../persistence/pagemem/NoOpWALManager.java     |    9 +-
 .../pagemem/PageMemoryImplNoLoadTest.java       |   23 +-
 .../persistence/pagemem/PageMemoryImplTest.java |    8 +
 .../wal/memtracker/PageMemoryTracker.java       |   13 +
 .../StandaloneWalRecordsIteratorTest.java       |    3 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |   66 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |   74 +-
 .../continuous/CacheContinuousBatchAckTest.java |   51 +
 ...eContinuousQueryAsyncFilterListenerTest.java |  165 +-
 ...nuousQueryConcurrentPartitionUpdateTest.java |  124 +-
 ...acheContinuousQueryExecuteInPrimaryTest.java |   33 +
 ...usQueryFactoryFilterRandomOperationTest.java |   58 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |    2 +-
 .../CacheContinuousQueryLostPartitionTest.java  |   26 +-
 .../CacheContinuousQueryOperationP2PTest.java   |   50 +-
 .../CacheContinuousQueryOrderingEventTest.java  |  137 +-
 ...acheContinuousQueryRandomOperationsTest.java |  271 +-
 ...inuousWithTransformerReplicatedSelfTest.java |    9 +
 ...CacheKeepBinaryIterationNearEnabledTest.java |    9 +
 ...acheKeepBinaryIterationStoreEnabledTest.java |   10 +
 .../CacheKeepBinaryIterationTest.java           |   26 +
 .../ClientReconnectContinuousQueryTest.java     |   14 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |  109 +-
 .../GridCacheContinuousQueryConcurrentTest.java |   65 +-
 ...eContinuousQueryMultiNodesFilteringTest.java |   18 +-
 ...eContinuousQueryReplicatedTxOneNodeTest.java |   15 +-
 ...niteCacheContinuousQueryBackupQueueTest.java |    8 +
 .../IgniteCacheContinuousQueryClientTest.java   |   10 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |   10 +-
 ...IgniteCacheContinuousQueryReconnectTest.java |    2 +-
 .../AbstractTransactionIntergrityTest.java      |  111 +-
 ...IntegrityWithPrimaryIndexCorruptionTest.java |  268 +-
 ...ctionIntegrityWithSystemWorkerDeathTest.java |    6 +-
 .../TxDataConsistencyOnCommitFailureTest.java   |  234 ++
 .../transactions/TxRollbackOnTimeoutTest.java   |    7 +-
 .../database/CacheFreeListImplSelfTest.java     |    5 -
 .../IgniteDbMemoryLeakLargeObjectsTest.java     |    2 +-
 .../DataStreamProcessorMvccSelfTest.java        |   40 +-
 .../processors/igfs/IgfsIgniteMock.java         |    8 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java |    2 +-
 .../GridServiceProxyNodeStopSelfTest.java       |    2 +-
 .../internal/util/IgniteUtilsSelfTest.java      |   74 +
 .../loadtests/hashmap/GridCacheTestContext.java |    1 +
 ...unicationSpiTcpFailureDetectionSelfTest.java |    2 +-
 .../TcpCommunicationSpiFaultyClientTest.java    |   37 +-
 .../tcp/TcpCommunicationStatisticsTest.java     |    4 +-
 .../LongClientConnectToClusterTest.java         |  173 ++
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |    4 +-
 .../tcp/TcpDiscoveryWithWrongServerTest.java    |    4 +-
 .../KeystoreEncryptionSpiSelfTest.java          |  123 +
 .../stream/socket/SocketStreamerSelfTest.java   |    2 +-
 .../ignite/testframework/GridStringLogger.java  |    3 +
 .../ignite/testframework/GridTestUtils.java     |   92 +-
 .../testframework/ListeningTestLogger.java      |  205 ++
 .../ignite/testframework/LogListener.java       |  427 +++
 .../config/GridTestProperties.java              |   10 +-
 .../testframework/junits/GridAbstractTest.java  |   90 +-
 .../junits/IgniteTestResources.java             |   14 +-
 .../junits/common/GridCommonAbstractTest.java   |   23 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    2 +-
 .../test/ListeningTestLoggerTest.java           |  428 +++
 .../ignite/testsuites/IgniteBasicTestSuite.java |    3 +
 .../IgniteBasicWithPersistenceTestSuite.java    |   17 +
 ...eBlockExchangeOnReadOperationsTestSuite.java |   51 +
 .../testsuites/IgniteCacheMvccTestSuite.java    |    3 +
 .../testsuites/IgniteCacheTestSuite4.java       |    3 +
 .../testsuites/IgniteCacheTestSuite5.java       |    5 +-
 .../testsuites/IgniteCacheTestSuite7.java       |    2 +-
 .../testsuites/IgniteCacheTestSuite9.java       |    3 +
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    2 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |    1 -
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |    8 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |    2 +
 .../ignite/testsuites/IgniteSpiTestSuite.java   |    3 +
 .../testsuites/IgniteUtilSelfTestSuite.java     |    7 +-
 .../ignite/util/GridLongListSelfTest.java       |   23 +
 ...gniteTaskTrackingThreadPoolExecutorTest.java |  140 -
 .../util/mbeans/GridMBeanDisableSelfTest.java   |   16 +-
 .../src/test/resources/other_tde_keystore.jks   |  Bin 0 -> 347 bytes
 modules/core/src/test/resources/tde.jks         |  Bin 0 -> 347 bytes
 .../testsuites/IgnitePdsNativeIoTestSuite2.java |    1 +
 .../apache/ignite/stream/flume/IgniteSink.java  |    4 +-
 .../ignite/stream/flume/IgniteSinkTest.java     |    2 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |    2 +-
 .../processors/hadoop/HadoopHelperImpl.java     |    8 +-
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |    4 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |    2 +-
 .../impl/HadoopAbstractMapReduceTest.java       |   18 +
 .../hadoop/impl/HadoopCommandLineTest.java      |   26 +
 .../taskexecutor/HadoopExecutorServiceTest.java |   52 +-
 modules/hibernate-4.2/pom.xml                   |   17 +
 modules/hibernate-5.1/pom.xml                   |   30 +
 .../processors/query/h2/H2TableDescriptor.java  |    5 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   15 +-
 .../query/h2/PreparedStatementExImpl.java       |    6 +-
 .../processors/query/h2/UpdateResult.java       |    3 +-
 .../processors/query/h2/database/H2Tree.java    |  145 +-
 .../query/h2/database/H2TreeFilterClosure.java  |   19 +-
 .../query/h2/database/H2TreeIndex.java          |   73 +-
 .../query/h2/database/InlineIndexHelper.java    |  143 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |    3 +-
 .../processors/query/h2/dml/DmlArguments.java   |    2 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |    2 +-
 .../processors/query/h2/opt/GridH2Row.java      |    5 -
 .../query/h2/sql/GridSqlCreateTable.java        |   17 +
 .../processors/query/h2/sql/GridSqlKeyword.java |    2 +-
 .../query/h2/sql/GridSqlQueryParser.java        |    8 +
 .../query/h2/sql/GridSqlQuerySplitter.java      |  137 +-
 .../h2/sys/view/SqlAbstractLocalSystemView.java |   13 +-
 .../query/h2/sys/view/SqlSystemViewCaches.java  |  197 ++
 .../h2/sys/view/SqlSystemViewNodeMetrics.java   |    2 +-
 .../query/h2/sys/view/SqlSystemViewNodes.java   |    2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |    5 +-
 ...MetadataConcurrentUpdateWithIndexesTest.java |  439 ++++
 .../cache/ClusterReadOnlyModeSqlTest.java       |   94 +
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |    4 +-
 .../cache/SqlFieldsQuerySelfTest.java           |    2 +-
 .../CacheBlockOnCreateDestoryIndexTest.java     |  480 ++++
 .../distributed/CacheBlockOnSqlQueryTest.java   |  131 +
 .../cache/encryption/EncryptedSqlTableTest.java |   69 +
 .../cache/index/DynamicColumnsAbstractTest.java |    6 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |   10 +-
 .../index/H2DynamicIndexAbstractSelfTest.java   |    2 +-
 .../H2DynamicIndexingComplexAbstractTest.java   |    8 +-
 .../cache/index/H2DynamicTableSelfTest.java     |    4 +-
 .../index/MvccEmptyTransactionSelfTest.java     |  110 +
 ...sactionsCommandsWithMvccEnabledSelfTest.java |   31 +-
 .../cache/index/SqlTransactionsSelfTest.java    |    4 +-
 ...acheMvccAbstractContinuousQuerySelfTest.java |   63 +
 ...eMvccAbstractSqlContinuousQuerySelfTest.java |   40 +
 .../mvcc/CacheMvccBasicContinuousQueryTest.java |  230 ++
 ...eMvccClientReconnectContinuousQueryTest.java |   30 +
 ...CacheMvccContinuousQueryBackupQueueTest.java |   30 +
 ...eMvccContinuousQueryClientReconnectTest.java |   32 +
 .../CacheMvccContinuousQueryClientTest.java     |   30 +
 ...heMvccContinuousQueryImmutableEntryTest.java |   30 +
 ...cContinuousQueryMultiNodesFilteringTest.java |   30 +
 ...eMvccContinuousQueryPartitionedSelfTest.java |   29 +
 ...ContinuousQueryPartitionedTxOneNodeTest.java |   36 +
 ...heMvccContinuousQueryReplicatedSelfTest.java |   29 +
 ...cContinuousQueryReplicatedTxOneNodeTest.java |   37 +
 ...ContinuousWithTransformerClientSelfTest.java |   35 +
 ...nuousWithTransformerPartitionedSelfTest.java |   42 +
 ...inuousWithTransformerReplicatedSelfTest.java |   36 +
 ...ccPartitionedSqlCoordinatorFailoverTest.java |    9 +
 ...cPartitionedSqlTxQueriesWithReducerTest.java |   49 +
 ...cheMvccSelectForUpdateQueryAbstractTest.java |   15 +-
 .../cache/mvcc/CacheMvccSizeTest.java           |    4 +-
 ...ccSqlContinuousQueryPartitionedSelfTest.java |   30 +
 ...vccSqlContinuousQueryReplicatedSelfTest.java |   29 +
 .../cache/mvcc/CacheMvccSqlLockTimeoutTest.java |    2 +-
 .../mvcc/CacheMvccSqlQueriesAbstractTest.java   |    4 +-
 .../mvcc/CacheMvccSqlTxQueriesAbstractTest.java |   26 +-
 ...MvccSqlTxQueriesWithReducerAbstractTest.java |    2 +-
 .../cache/mvcc/CacheMvccTxNodeMappingTest.java  |  213 ++
 .../cache/mvcc/CacheMvccTxRecoveryTest.java     |  654 +++++
 .../mvcc/MvccRepeatableReadBulkOpsTest.java     |  261 +-
 .../mvcc/MvccRepeatableReadOperationsTest.java  |   42 +-
 .../query/IgniteCachelessQueriesSelfTest.java   |    2 +-
 .../query/SqlSystemViewsSelfTest.java           |  225 ++
 ...GridIndexRebuildWithMvccEnabledSelfTest.java |    3 +-
 .../h2/database/InlineIndexHelperTest.java      |   30 +-
 .../twostep/CreateTableWithDateKeySelfTest.java |  236 ++
 .../InOperationExtractPartitionSelfTest.java    |  201 ++
 .../query/h2/twostep/JoinSqlTestHelper.java     |    9 +-
 .../NonCollocatedRetryMessageSelfTest.java      |  119 +-
 .../org/apache/ignite/sqltests/BaseSqlTest.java |   14 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |    3 +
 ...ockExchangeOnSqlReadOperationsTestSuite.java |   39 +
 .../testsuites/IgniteCacheMvccSqlTestSuite.java |  103 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    6 +
 .../IgniteCacheQuerySelfTestSuite2.java         |    3 +
 .../IgniteCacheQuerySelfTestSuite3.java         |   15 +-
 .../IgniteCacheWithIndexingTestSuite.java       |    3 +
 modules/indexing/src/test/resources/tde.jks     |  Bin 0 -> 347 bytes
 modules/kafka/pom.xml                           |   14 +
 .../ignite/stream/kafka/KafkaStreamer.java      |    2 +-
 .../kafka/KafkaIgniteStreamerSelfTest.java      |   22 +-
 .../ignite/stream/kafka/TestKafkaBroker.java    |    9 +-
 .../kafka/connect/IgniteSinkConnectorTest.java  |   18 +-
 .../connect/IgniteSourceConnectorTest.java      |   10 +-
 .../ignite/mesos/resource/IgniteProvider.java   |    6 +-
 .../main/java/org/apache/ignite/ml/Model.java   |    2 +-
 .../org/apache/ignite/ml/dataset/Dataset.java   |   22 +-
 .../ml/dataset/PartitionContextBuilder.java     |    2 +-
 .../ignite/ml/dataset/PartitionDataBuilder.java |    2 +-
 .../ml/environment/logging/NoOpLogger.java      |    2 +-
 .../java/org/apache/ignite/ml/math/Blas.java    |    4 +-
 .../java/org/apache/ignite/ml/math/Tracer.java  |   12 +-
 .../math/primitives/matrix/MatrixStorage.java   |    2 +-
 .../RPropUpdateCalculator.java                  |    4 +-
 .../ml/tree/impurity/ImpurityMeasure.java       |    2 +-
 .../impurity/util/StepFunctionCompressor.java   |    2 +-
 .../vector/VectorImplementationsFixtures.java   |    2 +-
 modules/platforms/cpp/core/namespaces.dox       |    2 +-
 modules/platforms/cpp/examples/Makefile.am      |    1 +
 modules/platforms/cpp/examples/configure.ac     |    1 +
 .../cpp/examples/project/vs/ignite-examples.sln |    6 +
 .../thin-client-put-get-example/Makefile.am     |   53 +
 .../vs/thin-client-put-get-example.vcxproj      |  107 +
 .../thin-client-put-get-example.vcxproj.filters |   35 +
 .../src/thin_client_put_get_example.cpp         |  127 +
 .../cpp/odbc-test/include/odbc_test_suite.h     |   11 +-
 .../cpp/odbc-test/include/test_utils.h          |   59 +
 .../cpp/odbc-test/src/meta_queries_test.cpp     |   34 +
 .../cpp/odbc-test/src/odbc_test_suite.cpp       |   28 +
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |   15 +
 .../cpp/odbc-test/src/transaction_test.cpp      |  244 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |   14 +-
 modules/platforms/cpp/odbc/src/common_types.cpp |    8 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |   12 +
 .../thin-client-test/src/cache_client_test.cpp  |  409 +++
 .../ignite/impl/thin/cache/cache_client_proxy.h |   59 +-
 .../include/ignite/impl/thin/readable.h         |   70 +-
 .../include/ignite/impl/thin/writable.h         |  155 +-
 .../include/ignite/thin/cache/cache_client.h    |  163 +-
 .../src/impl/cache/cache_client_impl.cpp        |  111 +-
 .../src/impl/cache/cache_client_impl.h          |   69 +-
 .../src/impl/cache/cache_client_proxy.cpp       |   33 +-
 .../cpp/thin-client/src/impl/message.cpp        |   20 +-
 .../cpp/thin-client/src/impl/message.h          |   47 +-
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |    4 +-
 .../Apache.Ignite.Core.Tests.DotNetCore/tde.jks |  Bin 0 -> 347 bytes
 .../ApiParity/IgniteConfigurationParityTest.cs  |    5 +-
 .../Serializable/AdvancedSerializationTest.cs   |    8 +-
 .../Binary/Serializable/SqlDmlTest.cs           |   20 +-
 .../Cache/CacheAbstractTest.cs                  |   45 +-
 .../Cache/DataRegionMetricsTest.cs              |    2 +-
 .../Cache/MemoryMetricsTest.cs                  |    2 +-
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   |   10 +-
 .../Client/Cache/CacheTestSsl.cs                |    2 +-
 .../Client/Cache/ScanQueryTest.cs               |    4 +-
 .../Client/IgniteClientConfigurationTest.cs     |    2 +-
 .../Compute/ComputeApiTest.JavaTask.cs          |    2 +-
 .../ConsoleRedirectTest.cs                      |   12 +-
 .../IgniteConfigurationTest.cs                  |   20 +
 .../Services/ServicesTest.cs                    |    2 +-
 .../Apache.Ignite.Core.csproj                   |    6 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   13 +
 .../Configuration/DataStorageConfiguration.cs   |    7 +
 .../Encryption/IEncryptionSpi.cs                |   34 +
 .../Keystore/KeystoreEncryptionSpi.cs           |   84 +
 .../Encryption/Keystore/Package-Info.cs         |   26 +
 .../Encryption/Package-Info.cs                  |   26 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   45 +
 .../IgniteConfigurationSection.xsd              |   53 +-
 .../Impl/Unmanaged/Jni/AppDomains.cs            |    8 +-
 .../Impl/Unmanaged/Jni/Jvm.cs                   |    2 +-
 .../Impl/Unmanaged/Jni/JvmDll.cs                |    8 +-
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   10 +-
 modules/platforms/php/.gitignore                |    2 +
 modules/platforms/php/README.md                 |   37 +
 modules/platforms/php/api_docs/Doxyfile         | 2487 ++++++++++++++++++
 modules/platforms/php/composer.json             |   27 +
 .../platforms/php/examples/AuthTlsExample.php   |  129 +
 .../php/examples/CachePutGetExample.php         |  184 ++
 .../platforms/php/examples/FailoverExample.php  |   67 +
 modules/platforms/php/examples/SqlExample.php   |  237 ++
 .../php/examples/SqlQueryEntriesExample.php     |  127 +
 modules/platforms/php/examples/certs/ca.pem     |   32 +
 modules/platforms/php/examples/certs/client.pem |   81 +
 .../platforms/php/examples/certs/keystore.jks   |  Bin 0 -> 3828 bytes
 .../platforms/php/examples/certs/truststore.jks |  Bin 0 -> 1477 bytes
 .../Apache/Ignite/Cache/CacheConfiguration.php  | 1011 +++++++
 .../php/src/Apache/Ignite/Cache/CacheEntry.php  |   60 +
 .../src/Apache/Ignite/Cache/CacheInterface.php  |  379 +++
 .../Ignite/Cache/CacheKeyConfiguration.php      |  107 +
 .../php/src/Apache/Ignite/Cache/QueryEntity.php |  315 +++
 .../php/src/Apache/Ignite/Cache/QueryField.php  |  279 ++
 .../php/src/Apache/Ignite/Cache/QueryIndex.php  |  191 ++
 .../platforms/php/src/Apache/Ignite/Client.php  |  243 ++
 .../src/Apache/Ignite/ClientConfiguration.php   |  294 +++
 .../php/src/Apache/Ignite/Data/BinaryObject.php |  469 ++++
 .../php/src/Apache/Ignite/Data/Date.php         |   83 +
 .../php/src/Apache/Ignite/Data/EnumItem.php     |  155 ++
 .../php/src/Apache/Ignite/Data/Time.php         |   58 +
 .../php/src/Apache/Ignite/Data/Timestamp.php    |   66 +
 .../Apache/Ignite/Exception/ClientException.php |   35 +
 .../Ignite/Exception/NoConnectionException.php  |   35 +
 .../Ignite/Exception/OperationException.php     |   35 +
 .../OperationStatusUnknownException.php         |   35 +
 .../Internal/Binary/BinaryCommunicator.php      |  490 ++++
 .../Ignite/Internal/Binary/BinaryField.php      |   78 +
 .../Internal/Binary/BinaryObjectField.php       |  113 +
 .../Ignite/Internal/Binary/BinarySchema.php     |  145 +
 .../Ignite/Internal/Binary/BinaryType.php       |  233 ++
 .../Internal/Binary/BinaryTypeBuilder.php       |  207 ++
 .../Internal/Binary/BinaryTypeStorage.php       |  123 +
 .../Ignite/Internal/Binary/BinaryUtils.php      |  438 +++
 .../Ignite/Internal/Binary/ClientOperation.php  |   64 +
 .../Ignite/Internal/Binary/MessageBuffer.php    |  307 +++
 .../Apache/Ignite/Internal/Binary/Request.php   |   85 +
 .../Apache/Ignite/Internal/Binary/TypeInfo.php  |  312 +++
 .../php/src/Apache/Ignite/Internal/Cache.php    |  387 +++
 .../Connection/ClientFailoverSocket.php         |  134 +
 .../Ignite/Internal/Connection/ClientSocket.php |  247 ++
 .../Internal/Connection/ProtocolVersion.php     |   82 +
 .../src/Apache/Ignite/Internal/Query/Cursor.php |  166 ++
 .../Ignite/Internal/Query/SqlFieldsCursor.php   |   75 +
 .../Ignite/Internal/Utils/ArgumentChecker.php   |   87 +
 .../src/Apache/Ignite/Internal/Utils/Logger.php |   62 +
 .../src/Apache/Ignite/Query/CursorInterface.php |   56 +
 .../php/src/Apache/Ignite/Query/Query.php       |   70 +
 .../php/src/Apache/Ignite/Query/ScanQuery.php   |   88 +
 .../Ignite/Query/SqlFieldsCursorInterface.php   |   82 +
 .../src/Apache/Ignite/Query/SqlFieldsQuery.php  |  206 ++
 .../php/src/Apache/Ignite/Query/SqlQuery.php    |  225 ++
 .../Apache/Ignite/Type/CollectionObjectType.php |  142 +
 .../Apache/Ignite/Type/ComplexObjectType.php    |  165 ++
 .../src/Apache/Ignite/Type/MapObjectType.php    |  123 +
 .../src/Apache/Ignite/Type/ObjectArrayType.php  |   68 +
 .../php/src/Apache/Ignite/Type/ObjectType.php   |  442 ++++
 .../platforms/php/tests/BinaryObjectTest.php    |  196 ++
 .../php/tests/CacheKeyValueOpsTest.php          |  763 ++++++
 modules/platforms/php/tests/CachePutGetTest.php |  646 +++++
 modules/platforms/php/tests/CacheTest.php       |  240 ++
 .../platforms/php/tests/ComplexObjectTest.php   |  428 +++
 modules/platforms/php/tests/ScanQueryTest.php   |  167 ++
 .../platforms/php/tests/SqlFieldsQueryTest.php  |  200 ++
 modules/platforms/php/tests/SqlQueryTest.php    |  204 ++
 modules/platforms/php/tests/TestConfig.php      |   37 +
 modules/platforms/php/tests/TestingHelper.php   |  363 +++
 .../tests/examples/ExecuteAuthTlsExample.php    |   36 +
 .../php/tests/examples/ExecuteExamples.php      |   61 +
 modules/platforms/python/LICENSE                |  202 ++
 modules/platforms/python/README.md              |   75 +
 modules/platforms/python/docs/Makefile          |   20 +
 modules/platforms/python/docs/conf.py           |  176 ++
 .../python/docs/datatypes/cache_props.rst       |  163 ++
 .../platforms/python/docs/datatypes/parsers.rst |  175 ++
 modules/platforms/python/docs/examples.rst      |  624 +++++
 modules/platforms/python/docs/index.rst         |   33 +
 modules/platforms/python/docs/modules.rst       |   31 +
 modules/platforms/python/docs/readme.rst        |  202 ++
 .../platforms/python/docs/source/modules.rst    |    7 +
 .../python/docs/source/pyignite.api.binary.rst  |    7 +
 .../docs/source/pyignite.api.cache_config.rst   |    7 +
 .../docs/source/pyignite.api.key_value.rst      |    7 +
 .../python/docs/source/pyignite.api.result.rst  |    7 +
 .../python/docs/source/pyignite.api.rst         |   19 +
 .../python/docs/source/pyignite.api.sql.rst     |    7 +
 .../python/docs/source/pyignite.binary.rst      |    7 +
 .../python/docs/source/pyignite.cache.rst       |    7 +
 .../python/docs/source/pyignite.client.rst      |    7 +
 .../source/pyignite.connection.generators.rst   |    7 +
 .../source/pyignite.connection.handshake.rst    |    7 +
 .../python/docs/source/pyignite.connection.rst  |   17 +
 .../docs/source/pyignite.connection.ssl.rst     |    7 +
 .../python/docs/source/pyignite.constants.rst   |    7 +
 .../docs/source/pyignite.datatypes.binary.rst   |    7 +
 .../source/pyignite.datatypes.cache_config.rst  |    7 +
 .../pyignite.datatypes.cache_properties.rst     |    7 +
 .../docs/source/pyignite.datatypes.complex.rst  |    7 +
 .../docs/source/pyignite.datatypes.internal.rst |    7 +
 .../source/pyignite.datatypes.key_value.rst     |    7 +
 .../source/pyignite.datatypes.null_object.rst   |    7 +
 .../source/pyignite.datatypes.primitive.rst     |    7 +
 .../pyignite.datatypes.primitive_arrays.rst     |    7 +
 .../pyignite.datatypes.primitive_objects.rst    |    7 +
 .../source/pyignite.datatypes.prop_codes.rst    |    7 +
 .../python/docs/source/pyignite.datatypes.rst   |   28 +
 .../docs/source/pyignite.datatypes.sql.rst      |    7 +
 .../docs/source/pyignite.datatypes.standard.rst |    7 +
 .../source/pyignite.datatypes.type_codes.rst    |    7 +
 .../python/docs/source/pyignite.exceptions.rst  |    7 +
 .../docs/source/pyignite.queries.op_codes.rst   |    7 +
 .../python/docs/source/pyignite.queries.rst     |   15 +
 .../platforms/python/docs/source/pyignite.rst   |   30 +
 .../python/docs/source/pyignite.utils.rst       |    7 +
 .../platforms/python/examples/binary_basics.py  |   53 +
 .../platforms/python/examples/create_binary.py  |  103 +
 modules/platforms/python/examples/failover.py   |   61 +
 .../platforms/python/examples/get_and_put.py    |   41 +
 .../platforms/python/examples/migrate_binary.py |  190 ++
 .../platforms/python/examples/read_binary.py    |  275 ++
 modules/platforms/python/examples/readme.md     |   17 +
 modules/platforms/python/examples/scans.py      |   55 +
 modules/platforms/python/examples/sql.py        |  298 +++
 modules/platforms/python/examples/type_hints.py |   51 +
 modules/platforms/python/pyignite/__init__.py   |   17 +
 .../platforms/python/pyignite/api/__init__.py   |   71 +
 modules/platforms/python/pyignite/api/binary.py |  209 ++
 .../python/pyignite/api/cache_config.py         |  279 ++
 .../platforms/python/pyignite/api/key_value.py  |  995 +++++++
 modules/platforms/python/pyignite/api/result.py |   38 +
 modules/platforms/python/pyignite/api/sql.py    |  478 ++++
 modules/platforms/python/pyignite/binary.py     |  136 +
 modules/platforms/python/pyignite/cache.py      |  595 +++++
 modules/platforms/python/pyignite/client.py     |  406 +++
 .../python/pyignite/connection/__init__.py      |  329 +++
 .../python/pyignite/connection/generators.py    |   48 +
 .../python/pyignite/connection/handshake.py     |   91 +
 .../platforms/python/pyignite/connection/ssl.py |   39 +
 modules/platforms/python/pyignite/constants.py  |   52 +
 .../python/pyignite/datatypes/__init__.py       |   27 +
 .../python/pyignite/datatypes/binary.py         |   45 +
 .../python/pyignite/datatypes/cache_config.py   |  153 ++
 .../pyignite/datatypes/cache_properties.py      |  287 ++
 .../python/pyignite/datatypes/complex.py        |  531 ++++
 .../python/pyignite/datatypes/internal.py       |  461 ++++
 .../python/pyignite/datatypes/key_value.py      |   24 +
 .../python/pyignite/datatypes/null_object.py    |   63 +
 .../python/pyignite/datatypes/primitive.py      |  105 +
 .../pyignite/datatypes/primitive_arrays.py      |  207 ++
 .../pyignite/datatypes/primitive_objects.py     |  157 ++
 .../python/pyignite/datatypes/prop_codes.py     |   51 +
 .../platforms/python/pyignite/datatypes/sql.py  |   23 +
 .../python/pyignite/datatypes/standard.py       |  729 +++++
 .../python/pyignite/datatypes/type_codes.py     |   57 +
 modules/platforms/python/pyignite/exceptions.py |   80 +
 .../python/pyignite/queries/__init__.py         |  339 +++
 .../python/pyignite/queries/op_codes.py         |   65 +
 modules/platforms/python/pyignite/utils.py      |  168 ++
 modules/platforms/python/requirements/docs.txt  |    6 +
 .../platforms/python/requirements/install.txt   |    4 +
 modules/platforms/python/requirements/setup.txt |    3 +
 modules/platforms/python/requirements/tests.txt |    5 +
 modules/platforms/python/setup.py               |  104 +
 modules/platforms/python/tests/conftest.py      |  218 ++
 modules/platforms/python/tests/test_binary.py   |  281 ++
 .../platforms/python/tests/test_cache_class.py  |  221 ++
 .../python/tests/test_cache_class_sql.py        |  103 +
 .../platforms/python/tests/test_cache_config.py |   75 +
 .../platforms/python/tests/test_datatypes.py    |  134 +
 modules/platforms/python/tests/test_examples.py |   57 +
 .../python/tests/test_generic_object.py         |   33 +
 .../platforms/python/tests/test_get_names.py    |   30 +
 .../platforms/python/tests/test_handshake.py    |   63 +
 .../platforms/python/tests/test_key_value.py    |  327 +++
 modules/platforms/python/tests/test_scan.py     |   66 +
 modules/platforms/python/tests/test_sql.py      |  154 ++
 .../rest/protocols/http/jetty/rest.html         |    4 +-
 .../ignite/spark/IgniteDataFrameSettings.scala  |   16 +-
 .../spark/impl/IgniteRelationProvider.scala     |   17 +-
 .../ignite/spark/impl/IgniteSQLRelation.scala   |   12 +-
 .../apache/ignite/spark/impl/QueryHelper.scala  |   13 +-
 .../org/apache/ignite/spark/impl/package.scala  |   95 +-
 .../sql/ignite/IgniteExternalCatalog.scala      |   92 +-
 .../spark/sql/ignite/IgniteOptimization.scala   |    3 +-
 .../ignite/spark/AbstractDataFrameSpec.scala    |    4 +-
 .../apache/ignite/spark/IgniteCatalogSpec.scala |   76 +-
 .../ignite/spark/IgniteSQLDataFrameSpec.scala   |   39 +
 .../spark/IgniteSQLDataFrameWriteSpec.scala     |   23 +-
 .../spring/src/test/config/enc/base-enc-cfg.xml |   70 +
 .../src/test/config/enc/enc-cache-client.xml    |   35 +
 .../spring/src/test/config/enc/enc-cache.xml    |   35 +
 .../spring/src/test/config/enc/enc-group-2.xml  |   36 +
 .../spring/src/test/config/enc/enc-group.xml    |   37 +
 .../config/enc/not-encrypted-cache-in-group.xml |   36 +
 .../src/test/config/enc/not-encrypted-cache.xml |   35 +
 .../SpringEncryptedCacheRestartClientTest.java  |   60 +
 .../SpringEncryptedCacheRestartTest.java        |  190 ++
 .../ignite/internal/GridFactorySelfTest.java    |    9 -
 .../testsuites/IgniteSpringTestSuite.java       |    6 +
 modules/spring/src/test/resources/tde.jks       |  Bin 0 -> 347 bytes
 modules/sqlline/bin/sqlline.bat                 |    6 +-
 modules/tensorflow/pom.xml                      |   30 +-
 modules/tensorflow/src/main/assembly/zip.xml    |   56 -
 .../TensorFlowServerScriptFormatter.java        |    3 +-
 .../tensorflow/src/main/resources/logback.xml   |   36 +
 modules/tensorflow/src/main/sh/ignite-tf.sh     |   19 -
 modules/tensorflow/src/main/sh/logback.xml      |   36 -
 modules/web-console/backend/app/agentSocket.js  |   23 +-
 .../web-console/backend/app/agentsHandler.js    |  145 +-
 .../web-console/backend/app/browsersHandler.js  |   51 +-
 modules/web-console/backend/app/configure.js    |   22 +-
 modules/web-console/backend/app/schemas.js      |    6 +-
 modules/web-console/backend/app/settings.js     |    3 +-
 .../e2e/testcafe/components/FormField.js        |   14 +
 .../e2e/testcafe/components/confirmation.js     |    4 +-
 modules/web-console/e2e/testcafe/package.json   |    2 +-
 .../web-console/e2e/testcafe/testcafe-runner.js |    3 +-
 modules/web-console/frontend/.eslintrc          |    1 -
 modules/web-console/frontend/app/app.config.js  |    3 +-
 modules/web-console/frontend/app/app.js         |    4 +-
 .../app/components/bs-select-menu/style.scss    |    2 +-
 .../components/cluster-selector/template.pug    |    2 +-
 .../components/cell-status/index.js             |   26 -
 .../components/cell-status/index.ts             |   31 +
 .../components/cell-status/style.scss           |   46 -
 .../components/cell-status/template.pug         |   18 -
 .../components/list/column-defs.js              |    2 +-
 .../app/components/ignite-status/index.js       |   22 -
 .../app/components/ignite-status/style.scss     |   27 -
 .../input-dialog/input-dialog.service.js        |  119 -
 .../input-dialog/input-dialog.service.ts        |  162 ++
 .../input-dialog/input-dialog.tpl.pug           |   36 +
 .../list-editable-cols/row.directive.js         |    5 +-
 .../cache-edit-form/templates/general.pug       |   18 +-
 .../cluster-edit-form/template.tpl.pug          |    3 +
 .../cluster-edit-form/templates/mvcc.pug        |   46 +
 .../page-configure-basic/controller.js          |    3 +-
 .../page-configure/reduxDevtoolsIntegration.js  |    5 +
 .../page-configure/services/ConfigureState.js   |    3 +-
 .../app/components/page-configure/style.scss    |    1 +
 .../app/components/page-landing/index.js        |    3 +-
 .../components/queries-notebook/style.scss      |    2 +-
 .../queries-notebooks-list/controller.js        |   64 +-
 .../components/page-queries/notebook.service.js |    2 +-
 .../app/components/status-output/component.ts   |   31 +
 .../status-output/componentFactory.ts           |   30 +
 .../app/components/status-output/controller.ts  |   53 +
 .../app/components/status-output/index.ts       |   40 +
 .../app/components/status-output/style.scss     |   38 +
 .../components/status-output/template.tpl.pug   |   18 +
 .../app/components/ui-grid-filters/template.pug |    3 +-
 .../app/components/ui-grid/controller.js        |    4 +
 .../components/web-console-header/component.js  |    4 +
 .../components/web-console-header/style.scss    |    3 +-
 .../components/web-console-header/template.pug  |    5 +-
 .../app/core/activities/Activities.data.js      |    5 +-
 modules/web-console/frontend/app/data/i18n.js   |  256 +-
 .../frontend/app/filters/duration.filter.js     |    6 +-
 .../app/modules/agent/AgentManager.service.js   |   27 +-
 .../frontend/app/modules/agent/types/Cluster.js |    2 +-
 .../generator/AbstractTransformer.js            |    5 +
 .../generator/ConfigurationGenerator.js         |   12 +-
 .../frontend/app/modules/user/User.service.js   |    7 +
 .../frontend/app/modules/user/user.module.js    |   21 +-
 .../app/primitives/datepicker/index.scss        |    1 +
 .../app/primitives/form-field/index.scss        |    2 +-
 .../app/primitives/timepicker/index.pug         |    5 +-
 .../app/primitives/timepicker/index.scss        |    1 +
 .../frontend/app/primitives/tooltip/index.scss  |    2 +-
 .../web-console/frontend/app/services/Caches.js |    3 +-
 .../frontend/app/services/FormUtils.service.js  |    2 +-
 .../frontend/app/services/exceptionHandler.js   |    4 +
 modules/web-console/frontend/package.json       |    5 +-
 .../stylesheets/_bootstrap-variables.scss       |    2 +-
 .../frontend/public/stylesheets/style.scss      |    1 -
 .../test/check-doc-links/check-doc-links.js     |    3 +-
 modules/web-console/frontend/views/index.pug    |    2 +
 .../web-agent/bin/ignite-web-agent.bat          |   63 +-
 .../web-agent/bin/ignite-web-agent.sh           |   81 +-
 .../web-agent/bin/include/functions.sh          |   88 +
 .../console/agent/handlers/ClusterListener.java |   20 +-
 .../ignite/console/agent/rest/RestExecutor.java |    2 +-
 .../ignite/cache/websession/WebSession.java     |    2 +-
 .../ignite/cache/websession/WebSessionV2.java   |    2 +-
 .../config/benchmark-multicast-tde.properties   |  128 +
 modules/yardstick/config/ignite-base-config.xml |   18 +-
 modules/yardstick/config/ignite-tde-config.xml  |   55 +
 .../mvcc/MvccUpdateContentionBenchmark.java     |    4 +-
 modules/yardstick/src/main/resources/tde.jks    |  Bin 0 -> 347 bytes
 .../org/apache/ignite/yarn/IgniteProvider.java  |    6 +-
 .../ZookeeperDiscoverySpiAbstractTestSuite.java |    2 +-
 parent/pom.xml                                  |   25 +-
 pom.xml                                         |    8 +-
 936 files changed, 60400 insertions(+), 6434 deletions(-)
----------------------------------------------------------------------



[35/50] [abbrv] ignite git commit: IGNITE-9950 Python: Fixes for Decimal data type

Posted by sb...@apache.org.
IGNITE-9950 Python: Fixes for Decimal data type

This closes #5041


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4bc08daa
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4bc08daa
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4bc08daa

Branch: refs/heads/ignite-gg-14206
Commit: 4bc08daa0cea486d9d2caa616707000dad5a56dc
Parents: d82b21e
Author: Dmitry Melnichuk <dm...@nobitlost.com>
Authored: Mon Oct 22 18:25:31 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Mon Oct 22 18:27:36 2018 +0300

----------------------------------------------------------------------
 .../python/pyignite/datatypes/standard.py       | 34 ++++++++++++++------
 modules/platforms/python/setup.py               |  2 +-
 2 files changed, 26 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4bc08daa/modules/platforms/python/pyignite/datatypes/standard.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/standard.py b/modules/platforms/python/pyignite/datatypes/standard.py
index 6109c7a..5f3af74 100644
--- a/modules/platforms/python/pyignite/datatypes/standard.py
+++ b/modules/platforms/python/pyignite/datatypes/standard.py
@@ -16,6 +16,7 @@
 import ctypes
 from datetime import date, datetime, time, timedelta
 import decimal
+from math import ceil
 import uuid
 
 from pyignite.constants import *
@@ -163,7 +164,7 @@ class DecimalObject:
             {
                 '_pack_': 1,
                 '_fields_': [
-                    ('data', ctypes.c_char * header.length),
+                    ('data', ctypes.c_ubyte * header.length),
                 ],
             }
         )
@@ -179,12 +180,18 @@ class DecimalObject:
             return None
 
         sign = 1 if ctype_object.data[0] & 0x80 else 0
-        data = bytes([ctype_object.data[0] & 0x7f]) + ctype_object.data[1:]
-        result = decimal.Decimal(data.decode(PROTOCOL_STRING_ENCODING))
+        data = ctype_object.data[1:]
+        data.insert(0, ctype_object.data[0] & 0x7f)
+        # decode n-byte integer
+        result = sum([
+            [x for x in reversed(data)][i] * 0x100 ** i for i in
+            range(len(data))
+        ])
         # apply scale
         result = (
             result
-            * decimal.Decimal('10') ** decimal.Decimal(ctype_object.scale)
+            / decimal.Decimal('10')
+            ** decimal.Decimal(ctype_object.scale)
         )
         if sign:
             # apply sign
@@ -197,12 +204,20 @@ class DecimalObject:
             return Null.from_python()
 
         sign, digits, scale = value.normalize().as_tuple()
-        data = bytearray([ord('0') + digit for digit in digits])
+        integer = int(''.join([str(d) for d in digits]))
+        # calculate number of bytes (at least one, and not forget the sign bit)
+        length = ceil((integer.bit_length() + 1)/8)
+        # write byte string
+        data = []
+        for i in range(length):
+            digit = integer % 0x100
+            integer //= 0x100
+            data.insert(0, digit)
+        # apply sign
         if sign:
             data[0] |= 0x80
         else:
             data[0] &= 0x7f
-        length = len(digits)
         header_class = cls.build_c_header()
         data_class = type(
             cls.__name__,
@@ -210,7 +225,7 @@ class DecimalObject:
             {
                 '_pack_': 1,
                 '_fields_': [
-                    ('data', ctypes.c_char * length),
+                    ('data', ctypes.c_ubyte * length),
                 ],
             }
         )
@@ -220,8 +235,9 @@ class DecimalObject:
             byteorder=PROTOCOL_BYTE_ORDER
         )
         data_object.length = length
-        data_object.scale = scale
-        data_object.data = bytes(data)
+        data_object.scale = -scale
+        for i in range(length):
+            data_object.data[i] = data[i]
         return bytes(data_object)
 
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4bc08daa/modules/platforms/python/setup.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/setup.py b/modules/platforms/python/setup.py
index 5d14dae..7419c97 100644
--- a/modules/platforms/python/setup.py
+++ b/modules/platforms/python/setup.py
@@ -65,7 +65,7 @@ for section in requirement_sections:
             if is_a_requirement(line):
                 requirements[section].append(line)
 
-with open('README.md', 'r') as readme_file:
+with open('README.md', 'r', encoding='utf-8') as readme_file:
     long_description = readme_file.read()
 
 setuptools.setup(


[39/50] [abbrv] ignite git commit: IGNITE-9960: SQL: Reverted IGNITE-9171 and IGNITE-9864 until performance is fixed. This closes #5045.

Posted by sb...@apache.org.
IGNITE-9960: SQL: Reverted IGNITE-9171 and IGNITE-9864 until performance is fixed. This closes #5045.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/75e414a4
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/75e414a4
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/75e414a4

Branch: refs/heads/ignite-gg-14206
Commit: 75e414a4ac53a36c421b31633723844faeb499af
Parents: 7504880
Author: devozerov <vo...@gridgain.com>
Authored: Tue Oct 23 10:33:26 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 23 10:33:26 2018 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |  12 +-
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |  38 +-
 .../jdbc/thin/JdbcThinDataSourceSelfTest.java   |  12 +-
 .../apache/ignite/IgniteSystemProperties.java   |   7 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |  20 +-
 .../jdbc/thin/ConnectionPropertiesImpl.java     |   2 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   2 +-
 .../query/h2/H2ConnectionWrapper.java           |  11 -
 .../internal/processors/query/h2/H2Utils.java   |  15 -
 .../processors/query/h2/IgniteH2Indexing.java   | 186 +++----
 .../processors/query/h2/ObjectPool.java         |  97 ----
 .../processors/query/h2/ObjectPoolReusable.java |  58 ---
 .../query/h2/ThreadLocalObjectPool.java         | 103 ++++
 .../processors/query/h2/dml/UpdatePlan.java     |   8 +-
 .../query/h2/opt/GridH2QueryContext.java        |  33 +-
 .../processors/query/h2/opt/GridH2Table.java    | 133 +----
 .../query/h2/twostep/GridMapQueryExecutor.java  | 498 +++++++++----------
 .../h2/twostep/GridReduceQueryExecutor.java     |  16 +-
 .../query/h2/twostep/GridResultPage.java        |   7 +-
 .../query/h2/twostep/MapNodeResults.java        |  13 +-
 .../query/h2/twostep/MapQueryLazyWorker.java    | 223 ++-------
 .../query/h2/twostep/MapQueryResult.java        |  34 +-
 .../query/h2/twostep/MapQueryResults.java       |  40 +-
 ...GridCacheLazyQueryPartitionsReleaseTest.java |   2 +
 .../IgniteCacheQueryH2IndexingLeakTest.java     |   9 +-
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |  43 +-
 ...cheQueryAbstractDistributedJoinSelfTest.java |   5 -
 ...QueryNodeRestartDistributedJoinSelfTest.java |  14 +-
 ...nCancelOrTimeoutDistributedJoinSelfTest.java |  23 +-
 ...ynamicColumnsAbstractConcurrentSelfTest.java |   6 +-
 .../cache/index/H2ConnectionLeaksSelfTest.java  |   2 +-
 .../processors/query/LazyQuerySelfTest.java     | 202 +-------
 .../processors/query/h2/ObjectPoolSelfTest.java | 125 -----
 .../query/h2/ThreadLocalObjectPoolSelfTest.java | 113 +++++
 .../h2/twostep/RetryCauseMessageSelfTest.java   |  16 +
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 .../ignite/cache/query/query_sql_fields.h       |   4 +-
 .../cpp/odbc-test/src/configuration_test.cpp    |   4 +-
 .../cpp/odbc/src/config/configuration.cpp       |   2 +-
 .../Cache/Query/CacheQueriesTest.cs             |   6 +-
 .../Query/Linq/CacheLinqTest.Introspection.cs   |   2 -
 .../Client/Cache/SqlQueryTest.cs                |   4 +-
 .../Cache/Query/SqlFieldsQuery.cs               |  43 +-
 ...benchmark-native-sql-cache-select.properties |  96 ----
 .../benchmark-native-sql-select.properties      |  17 +-
 .../ignite-localhost-sql-query-config.xml       |  91 ----
 .../yardstick/IgniteAbstractBenchmark.java      |  30 +-
 .../yardstick/IgniteBenchmarkArguments.java     |  13 -
 .../yardstick/jdbc/AbstractNativeBenchmark.java |   3 -
 .../apache/ignite/yardstick/jdbc/JdbcUtils.java |  47 +-
 .../jdbc/NativeSqlCacheQueryRangeBenchmark.java | 145 ------
 .../jdbc/NativeSqlQueryRangeBenchmark.java      |  13 +-
 53 files changed, 848 insertions(+), 1806 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
index db0a959..d560d74 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
@@ -308,7 +308,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
             assertTrue(((JdbcConnection)conn).isEnforceJoinOrder());
             assertFalse(((JdbcConnection)conn).isDistributedJoins());
             assertFalse(((JdbcConnection)conn).isCollocatedQuery());
-            assertTrue(((JdbcConnection)conn).isLazy());
+            assertFalse(((JdbcConnection)conn).isLazy());
             assertFalse(((JdbcConnection)conn).skipReducerOnUpdate());
         }
 
@@ -317,7 +317,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
             assertFalse(((JdbcConnection)conn).isEnforceJoinOrder());
             assertTrue(((JdbcConnection)conn).isDistributedJoins());
             assertFalse(((JdbcConnection)conn).isCollocatedQuery());
-            assertTrue(((JdbcConnection)conn).isLazy());
+            assertFalse(((JdbcConnection)conn).isLazy());
             assertFalse(((JdbcConnection)conn).skipReducerOnUpdate());
         }
 
@@ -326,15 +326,15 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
             assertFalse(((JdbcConnection)conn).isEnforceJoinOrder());
             assertFalse(((JdbcConnection)conn).isDistributedJoins());
             assertTrue(((JdbcConnection)conn).isCollocatedQuery());
-            assertTrue(((JdbcConnection)conn).isLazy());
+            assertFalse(((JdbcConnection)conn).isLazy());
             assertFalse(((JdbcConnection)conn).skipReducerOnUpdate());
         }
 
-        try (final Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + "lazy=false@" + configURL())) {
+        try (final Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + "lazy=true@" + configURL())) {
             assertFalse(((JdbcConnection)conn).isEnforceJoinOrder());
             assertFalse(((JdbcConnection)conn).isDistributedJoins());
             assertFalse(((JdbcConnection)conn).isCollocatedQuery());
-            assertFalse(((JdbcConnection)conn).isLazy());
+            assertTrue(((JdbcConnection)conn).isLazy());
             assertFalse(((JdbcConnection)conn).skipReducerOnUpdate());
         }
         try (final Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + "skipReducerOnUpdate=true@"
@@ -342,7 +342,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
             assertFalse(((JdbcConnection)conn).isEnforceJoinOrder());
             assertFalse(((JdbcConnection)conn).isDistributedJoins());
             assertFalse(((JdbcConnection)conn).isCollocatedQuery());
-            assertTrue(((JdbcConnection)conn).isLazy());
+            assertFalse(((JdbcConnection)conn).isLazy());
             assertTrue(((JdbcConnection)conn).skipReducerOnUpdate());
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
index 26c34cf..80397e6 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
@@ -230,36 +230,36 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
      */
     public void testSqlHints() throws Exception {
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1")) {
-            assertHints(conn, false, false, false, false, true, false);
+            assertHints(conn, false, false, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?distributedJoins=true")) {
-            assertHints(conn, true, false, false, false, true, false);
+            assertHints(conn, true, false, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?enforceJoinOrder=true")) {
-            assertHints(conn, false, true, false, false, true, false);
+            assertHints(conn, false, true, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?collocated=true")) {
-            assertHints(conn, false, false, true, false, true, false);
+            assertHints(conn, false, false, true, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?replicatedOnly=true")) {
-            assertHints(conn, false, false, false, true, true, false);
+            assertHints(conn, false, false, false, true, false, false);
         }
 
-        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?lazy=false")) {
-            assertHints(conn, false, false, false, false, false, false);
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?lazy=true")) {
+            assertHints(conn, false, false, false, false, true, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?skipReducerOnUpdate=true")) {
-            assertHints(conn, false, false, false, false, true, true);
+            assertHints(conn, false, false, false, false, false, true);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?distributedJoins=true&" +
-            "enforceJoinOrder=true&collocated=true&replicatedOnly=true&lazy=false&skipReducerOnUpdate=true")) {
-            assertHints(conn, true, true, true, true, false, true);
+            "enforceJoinOrder=true&collocated=true&replicatedOnly=true&lazy=true&skipReducerOnUpdate=true")) {
+            assertHints(conn, true, true, true, true, true, true);
         }
     }
 
@@ -270,32 +270,32 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
      */
     public void testSqlHintsSemicolon() throws Exception {
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;distributedJoins=true")) {
-            assertHints(conn, true, false, false, false, true, false);
+            assertHints(conn, true, false, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;enforceJoinOrder=true")) {
-            assertHints(conn, false, true, false, false, true, false);
+            assertHints(conn, false, true, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;collocated=true")) {
-            assertHints(conn, false, false, true, false, true, false);
+            assertHints(conn, false, false, true, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;replicatedOnly=true")) {
-            assertHints(conn, false, false, false, true, true, false);
+            assertHints(conn, false, false, false, true, false, false);
         }
 
-        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;lazy=false")) {
-            assertHints(conn, false, false, false, false, false, false);
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;lazy=true")) {
+            assertHints(conn, false, false, false, false, true, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;skipReducerOnUpdate=true")) {
-            assertHints(conn, false, false, false, false, true, true);
+            assertHints(conn, false, false, false, false, false, true);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;distributedJoins=true;" +
-            "enforceJoinOrder=true;collocated=true;replicatedOnly=true;lazy=false;skipReducerOnUpdate=true")) {
-            assertHints(conn, true, true, true, true, false, true);
+            "enforceJoinOrder=true;collocated=true;replicatedOnly=true;lazy=true;skipReducerOnUpdate=true")) {
+            assertHints(conn, true, true, true, true, true, true);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
index 834b4ca..6040bed 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
@@ -142,15 +142,15 @@ public class JdbcThinDataSourceSelfTest extends JdbcThinAbstractSelfTest {
     public void testResetUrl() throws Exception {
         IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
 
-        ids.setUrl("jdbc:ignite:thin://127.0.0.1:10800/test?lazy=false");
+        ids.setUrl("jdbc:ignite:thin://127.0.0.1:10800/test?lazy=true");
 
         assertEquals("test", ids.getSchema());
-        assertFalse(ids.isLazy());
+        assertTrue(ids.isLazy());
 
         ids.setUrl("jdbc:ignite:thin://mydomain.org,localhost?collocated=true");
 
         assertNull(ids.getSchema());
-        assertTrue(ids.isLazy());
+        assertFalse(ids.isLazy());
         assertTrue(ids.isCollocated());
     }
 
@@ -168,7 +168,7 @@ public class JdbcThinDataSourceSelfTest extends JdbcThinAbstractSelfTest {
             assertFalse(io.connectionProperties().isAutoCloseServerCursor());
             assertFalse(io.connectionProperties().isCollocated());
             assertFalse(io.connectionProperties().isEnforceJoinOrder());
-            assertTrue(io.connectionProperties().isLazy());
+            assertFalse(io.connectionProperties().isLazy());
             assertFalse(io.connectionProperties().isDistributedJoins());
             assertFalse(io.connectionProperties().isReplicatedOnly());
         }
@@ -176,7 +176,7 @@ public class JdbcThinDataSourceSelfTest extends JdbcThinAbstractSelfTest {
         ids.setAutoCloseServerCursor(true);
         ids.setCollocated(true);
         ids.setEnforceJoinOrder(true);
-        ids.setLazy(false);
+        ids.setLazy(true);
         ids.setDistributedJoins(true);
         ids.setReplicatedOnly(true);
 
@@ -186,7 +186,7 @@ public class JdbcThinDataSourceSelfTest extends JdbcThinAbstractSelfTest {
             assertTrue(io.connectionProperties().isAutoCloseServerCursor());
             assertTrue(io.connectionProperties().isCollocated());
             assertTrue(io.connectionProperties().isEnforceJoinOrder());
-            assertFalse(io.connectionProperties().isLazy());
+            assertTrue(io.connectionProperties().isLazy());
             assertTrue(io.connectionProperties().isDistributedJoins());
             assertTrue(io.connectionProperties().isReplicatedOnly());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 6afe244..ef416c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -484,12 +484,7 @@ public final class IgniteSystemProperties {
     /** Disable fallback to H2 SQL parser if the internal SQL parser fails to parse the statement. */
     public static final String IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK = "IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK";
 
-    /**
-     *  Force all SQL queries to be processed lazily regardless of what clients request.
-     *
-     * @deprecated Since version 2.7.
-     */
-    @Deprecated
+    /** Force all SQL queries to be processed lazily regardless of what clients request. */
     public static final String IGNITE_SQL_FORCE_LAZY_RESULT_SET = "IGNITE_SQL_FORCE_LAZY_RESULT_SET";
 
     /** Disable SQL system views. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index 3e5c706..4e12b8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
@@ -71,8 +71,8 @@ public class SqlFieldsQuery extends Query<List<?>> {
     /** */
     private boolean replicatedOnly;
 
-    /** Lazy mode is default since Ignite v.2.7. */
-    private boolean lazy = true;
+    /** */
+    private boolean lazy;
 
     /** Partitions for query */
     private int[] parts;
@@ -292,24 +292,19 @@ public class SqlFieldsQuery extends Query<List<?>> {
     /**
      * Sets lazy query execution flag.
      * <p>
+     * By default Ignite attempts to fetch the whole query result set to memory and send it to the client. For small
+     * and medium result sets this provides optimal performance and minimize duration of internal database locks, thus
+     * increasing concurrency.
+     * <p>
      * If result set is too big to fit in available memory this could lead to excessive GC pauses and even
      * OutOfMemoryError. Use this flag as a hint for Ignite to fetch result set lazily, thus minimizing memory
      * consumption at the cost of moderate performance hit.
-     * Now lazy mode is optimized for small and medium result set. Small result set means results rows count
-     * less then page size (see {@link #setPageSize}).
      * <p>
-     * To compatibility with previous version behavior lazy mode may be switched off. In this case Ignite attempts
-     * to fetch the whole query result set to memory and send it to the client.
-     * <p>
-     * Since version 2.7 lazy mode is used by default.
-     * Defaults to {@code true}, meaning that the result set is fetched lazily if it is possible.
+     * Defaults to {@code false}, meaning that the whole result set is fetched to memory eagerly.
      *
      * @param lazy Lazy query execution flag.
      * @return {@code this} For chaining.
-     *
-     * @deprecated Since Ignite 2.7.
      */
-    @Deprecated
     public SqlFieldsQuery setLazy(boolean lazy) {
         this.lazy = lazy;
 
@@ -323,7 +318,6 @@ public class SqlFieldsQuery extends Query<List<?>> {
      *
      * @return Lazy flag.
      */
-    @Deprecated
     public boolean isLazy() {
         return lazy;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
index 054807a..51a3837 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
@@ -84,7 +84,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
 
     /** Lazy query execution property. */
     private BooleanProperty lazy = new BooleanProperty(
-        "lazy", "Enable lazy query execution (lazy mode is used by default since v.2.7)", true, false);
+        "lazy", "Enable lazy query execution", false, false);
 
     /** Socket send buffer size property. */
     private IntegerProperty socketSendBuffer = new IntegerProperty(

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index 481794e..c589c06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -195,7 +195,7 @@ public class JdbcConnection implements Connection {
         collocatedQry = Boolean.parseBoolean(props.getProperty(PROP_COLLOCATED));
         distributedJoins = Boolean.parseBoolean(props.getProperty(PROP_DISTRIBUTED_JOINS));
         enforceJoinOrder = Boolean.parseBoolean(props.getProperty(PROP_ENFORCE_JOIN_ORDER));
-        lazy = Boolean.parseBoolean(props.getProperty(PROP_LAZY, "true"));
+        lazy = Boolean.parseBoolean(props.getProperty(PROP_LAZY));
         txAllowed = Boolean.parseBoolean(props.getProperty(PROP_TX_ALLOWED));
 
         stream = Boolean.parseBoolean(props.getProperty(PROP_STREAMING));

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 2fe0eb8..1e8d005 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
@@ -4069,7 +4069,7 @@ public abstract class IgniteUtils {
                 rsrc.close();
             }
             catch (Exception e) {
-                warn(log, "Failed to close resource: " + e.getMessage(), e);
+                warn(log, "Failed to close resource: " + e.getMessage());
             }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java
index 020cd5e..425015a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java
@@ -33,9 +33,6 @@ public class H2ConnectionWrapper implements AutoCloseable {
     private final Connection conn;
 
     /** */
-    private final Thread intiThread;
-
-    /** */
     private volatile String schema;
 
     /** */
@@ -46,7 +43,6 @@ public class H2ConnectionWrapper implements AutoCloseable {
      */
     H2ConnectionWrapper(Connection conn) {
         this.conn = conn;
-        intiThread = Thread.currentThread();
 
         initStatementCache();
     }
@@ -100,13 +96,6 @@ public class H2ConnectionWrapper implements AutoCloseable {
         statementCache = new H2StatementCache(STATEMENT_CACHE_SIZE);
     }
 
-    /**
-     * @return Thread where the connection was created.
-     */
-    public Thread initialThread() {
-        return intiThread;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(H2ConnectionWrapper.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
index 074a3e4..b9d9d8e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
@@ -237,25 +237,10 @@ public class H2Utils {
      * @param enforceJoinOrder Enforce join order of tables.
      */
     public static void setupConnection(Connection conn, boolean distributedJoins, boolean enforceJoinOrder) {
-        setupConnection(conn,distributedJoins, enforceJoinOrder, false);
-    }
-
-    /**
-     * @param conn Connection to use.
-     * @param distributedJoins If distributed joins are enabled.
-     * @param enforceJoinOrder Enforce join order of tables.
-     * @param lazy Lazy query execution mode.
-     */
-    public static void setupConnection(
-        Connection conn,
-        boolean distributedJoins,
-        boolean enforceJoinOrder,
-        boolean lazy) {
         Session s = session(conn);
 
         s.setForceJoinOrder(enforceJoinOrder);
         s.setJoinBatchEnabled(distributedJoins);
-        s.setLazyQueryExecution(lazy);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 255e2b1..cbe2d0b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -137,6 +137,7 @@ import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNode
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodes;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
+import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
@@ -298,9 +299,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** */
     private String dbUrl = "jdbc:h2:mem:";
 
-    /** All connections are used by Ignite instance. Map of (H2ConnectionWrapper, Boolean) is used as a Set. */
+    /** */
     // TODO https://issues.apache.org/jira/browse/IGNITE-9062
-    private final ConcurrentMap<Thread, ConcurrentHashMap<H2ConnectionWrapper, Boolean>> conns = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Thread, H2ConnectionWrapper> conns = new ConcurrentHashMap<>();
 
     /** */
     private GridMapQueryExecutor mapQryExec;
@@ -328,23 +329,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** */
     // TODO https://issues.apache.org/jira/browse/IGNITE-9062
-    private final ThreadLocal<ObjectPool<H2ConnectionWrapper>> connectionPool
-        = new ThreadLocal<ObjectPool<H2ConnectionWrapper>>() {
-        @Override protected ObjectPool<H2ConnectionWrapper> initialValue() {
-            return new ObjectPool<>(
-                IgniteH2Indexing.this::newConnectionWrapper,
-                50,
-                IgniteH2Indexing.this::closePooledConnectionWrapper,
-                IgniteH2Indexing.this::recycleConnection);
-        }
-    };
+    private final ThreadLocalObjectPool<H2ConnectionWrapper> connectionPool = new ThreadLocalObjectPool<>(IgniteH2Indexing.this::newConnectionWrapper, 5);
 
     /** */
     // TODO https://issues.apache.org/jira/browse/IGNITE-9062
-    private final ThreadLocal<ObjectPoolReusable<H2ConnectionWrapper>> connCache
-        = new ThreadLocal<ObjectPoolReusable<H2ConnectionWrapper>>() {
-        @Override public ObjectPoolReusable<H2ConnectionWrapper> get() {
-            ObjectPoolReusable<H2ConnectionWrapper> reusable = super.get();
+    private final ThreadLocal<ThreadLocalObjectPool.Reusable<H2ConnectionWrapper>> connCache = new ThreadLocal<ThreadLocalObjectPool.Reusable<H2ConnectionWrapper>>() {
+        @Override public ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> get() {
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusable = super.get();
 
             boolean reconnect = true;
 
@@ -364,21 +355,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return reusable;
         }
 
-        @Override protected ObjectPoolReusable<H2ConnectionWrapper> initialValue() {
-            ObjectPool<H2ConnectionWrapper> pool = connectionPool.get();
-
-            ObjectPoolReusable<H2ConnectionWrapper> reusableConnection = pool.borrow();
-
-            ConcurrentHashMap<H2ConnectionWrapper, Boolean> perThreadConns = conns.get(Thread.currentThread());
+        @Override protected ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> initialValue() {
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusableConnection = connectionPool.borrow();
 
-            ConcurrentHashMap<H2ConnectionWrapper, Boolean> newMap = new ConcurrentHashMap<>();
-
-            perThreadConns = conns.putIfAbsent(Thread.currentThread(), newMap);
-
-            if (perThreadConns == null)
-                perThreadConns = newMap;
-
-            perThreadConns.put(reusableConnection.object(), false);
+            conns.put(Thread.currentThread(), reusableConnection.object());
 
             return reusableConnection;
         }
@@ -458,54 +438,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return sysConn;
     }
 
-    /**
-     * @return Connection wrapper.
-     */
+    /** */
     private H2ConnectionWrapper newConnectionWrapper() {
         try {
-            Connection c = DriverManager.getConnection(dbUrl);
-            return new H2ConnectionWrapper(c);
+            return new H2ConnectionWrapper(DriverManager.getConnection(dbUrl));
         } catch (SQLException e) {
             throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
         }
     }
 
     /**
-     * @param conn Connection wrapper to close.
-     */
-    private void closePooledConnectionWrapper(H2ConnectionWrapper conn) {
-        conns.get(conn.initialThread()).remove(conn);
-
-        U.closeQuiet(conn);
-    }
-
-    /**
-     * Removes from threadlocal cache and returns associated with current thread connection.
-     * @return Connection associated with current thread.
-     */
-    public ObjectPoolReusable<H2ConnectionWrapper> detachConnection() {
-        ObjectPoolReusable<H2ConnectionWrapper> reusableConnection = connCache.get();
-
-        connCache.remove();
-
-        conns.get(Thread.currentThread()).remove(reusableConnection.object());
-
-        return reusableConnection;
-    }
-
-    /**
-     * Return connection to the glob all connection collection.
-     * @param conn Recycled connection.
-     */
-    private void recycleConnection(H2ConnectionWrapper conn) {
-        ConcurrentMap<H2ConnectionWrapper, Boolean> perThreadConns = conns.get(conn.initialThread());
-
-        // Mau be null when node is stopping.
-        if (perThreadConns != null)
-            perThreadConns.put(conn, false);
-    }
-
-    /**
      * @param c Connection.
      * @param sql SQL.
      * @return <b>Cached</b> prepared statement.
@@ -797,12 +739,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * Handles SQL exception.
      */
     private void onSqlException() {
-        H2ConnectionWrapper conn = connCache.get().object();
+        Connection conn = connCache.get().object().connection();
 
         connCache.set(null);
 
         if (conn != null) {
-            conns.get(Thread.currentThread()).remove(conn);
+            conns.remove(Thread.currentThread());
 
             // Reset connection to receive new one at next call.
             U.close(conn, log);
@@ -1449,15 +1391,31 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement stmt,
         int timeoutMillis, @Nullable GridQueryCancel cancel) throws IgniteCheckedException {
-        if (cancel != null)
-            cancel.set(() -> cancelStatement(stmt));
+        final MapQueryLazyWorker lazyWorker = MapQueryLazyWorker.currentWorker();
+
+        if (cancel != null) {
+            cancel.set(new Runnable() {
+                @Override public void run() {
+                    if (lazyWorker != null) {
+                        lazyWorker.submit(new Runnable() {
+                            @Override public void run() {
+                                cancelStatement(stmt);
+                            }
+                        });
+                    }
+                    else
+                        cancelStatement(stmt);
+                }
+            });
+        }
 
         Session ses = H2Utils.session(conn);
 
         if (timeoutMillis > 0)
             ses.setQueryTimeout(timeoutMillis);
-        else
-            ses.setQueryTimeout(0);
+
+        if (lazyWorker != null)
+            ses.setLazyQueryExecution(true);
 
         try {
             return stmt.executeQuery();
@@ -1469,6 +1427,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             throw new IgniteCheckedException("Failed to execute SQL query. " + e.getMessage(), e);
         }
+        finally {
+            if (timeoutMillis > 0)
+                ses.setQueryTimeout(0);
+
+            if (lazyWorker != null)
+                ses.setLazyQueryExecution(false);
+        }
     }
 
     /**
@@ -2579,11 +2544,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             topVer, mvccSnapshot);
     }
 
-    /**
-     * @param flags Flags holder.
-     * @param flag Flag mask to check.
-     * @return {@code true} if flag is set, otherwise returns {@code false}.
-     */
     private boolean isFlagSet(int flags, int flag) {
         return (flags & flag) == flag;
     }
@@ -3061,24 +3021,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private void cleanupStatementCache() {
         long now = U.currentTimeMillis();
 
-        for (Iterator<Map.Entry<Thread, ConcurrentHashMap<H2ConnectionWrapper, Boolean>>> it
-            = conns.entrySet().iterator(); it.hasNext(); ) {
-            Map.Entry<Thread, ConcurrentHashMap<H2ConnectionWrapper, Boolean>> entry = it.next();
+        for (Iterator<Map.Entry<Thread, H2ConnectionWrapper>> it = conns.entrySet().iterator(); it.hasNext(); ) {
+            Map.Entry<Thread, H2ConnectionWrapper> entry = it.next();
 
             Thread t = entry.getKey();
 
             if (t.getState() == Thread.State.TERMINATED) {
-                for (H2ConnectionWrapper c : entry.getValue().keySet())
-                    U.close(c, log);
+                U.close(entry.getValue(), log);
 
                 it.remove();
             }
-            else {
-                for (H2ConnectionWrapper c : entry.getValue().keySet()) {
-                    if (now - c.statementCache().lastUsage() > STATEMENT_CACHE_THREAD_USAGE_TIMEOUT)
-                        c.clearStatementCache();
-                }
-            }
+            else if (now - entry.getValue().statementCache().lastUsage() > STATEMENT_CACHE_THREAD_USAGE_TIMEOUT)
+                entry.getValue().clearStatementCache();
         }
     }
 
@@ -3086,15 +3040,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * Called periodically by {@link GridTimeoutProcessor} to clean up the {@link #conns}.
      */
     private void cleanupConnections() {
-        for (Iterator<Map.Entry<Thread, ConcurrentHashMap<H2ConnectionWrapper, Boolean>>> it
-            = conns.entrySet().iterator(); it.hasNext(); ) {
-            Map.Entry<Thread, ConcurrentHashMap<H2ConnectionWrapper, Boolean>> entry = it.next();
+        for (Iterator<Map.Entry<Thread, H2ConnectionWrapper>> it = conns.entrySet().iterator(); it.hasNext(); ) {
+            Map.Entry<Thread, H2ConnectionWrapper> entry = it.next();
 
             Thread t = entry.getKey();
 
             if (t.getState() == Thread.State.TERMINATED) {
-                for (H2ConnectionWrapper c : entry.getValue().keySet())
-                    U.close(c, log);
+                U.close(entry.getValue(), log);
 
                 it.remove();
             }
@@ -3102,6 +3054,24 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * Removes from cache and returns associated with current thread connection.
+     * @return Connection associated with current thread.
+     */
+    public ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> detach() {
+        Thread key = Thread.currentThread();
+
+        ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> reusableConnection = connCache.get();
+
+        H2ConnectionWrapper connection = conns.remove(key);
+
+        connCache.remove();
+
+        assert reusableConnection.object().connection() == connection.connection();
+
+        return reusableConnection;
+    }
+
+    /**
      * Rebuild indexes from hash index.
      *
      * @param cacheName Cache name.
@@ -3466,15 +3436,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Stopping cache query index...");
 
-        mapQryExec.stop();
-
-        for (ConcurrentMap<H2ConnectionWrapper, Boolean> perThreadConns : conns.values()) {
-            for (H2ConnectionWrapper c : perThreadConns.keySet())
-                U.close(c, log);
-        }
+        mapQryExec.cancelLazyWorkers();
 
-        connectionPool.remove();
-        connCache.remove();
+        for (H2ConnectionWrapper c : conns.values())
+            U.close(c, log);
 
         conns.clear();
         schemas.clear();
@@ -3583,7 +3548,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 }
             }
 
-            conns.values().forEach(map -> map.keySet().forEach(H2ConnectionWrapper::clearStatementCache));
+            conns.values().forEach(H2ConnectionWrapper::clearStatementCache);
 
             for (H2TableDescriptor tbl : rmvTbls) {
                 for (Index idx : tbl.table().getIndexes())
@@ -3741,10 +3706,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public void cancelAllQueries() {
-        for (ConcurrentHashMap<H2ConnectionWrapper, Boolean> perThreadConns : conns.values()) {
-            for (H2ConnectionWrapper c : perThreadConns.keySet())
-                U.close(c, log);
-        }
+        mapQryExec.cancelLazyWorkers();
+
+        for (H2ConnectionWrapper c : conns.values())
+            U.close(c, log);
     }
 
     /**
@@ -3794,7 +3759,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param twoStepQry Query.
      * @return {@code True} is system views exist.
      */
     private boolean hasSystemViews(GridCacheTwoStepQuery twoStepQry) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPool.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPool.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPool.java
deleted file mode 100644
index 9d2a580..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPool.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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.query.h2;
-
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-/**
- * Thread-safe pool for managing limited number objects for further reuse.
- *
- * @param <E> Pooled objects type.
- */
-public final class ObjectPool<E extends AutoCloseable> {
-    /** */
-    private final Supplier<E> objectFactory;
-
-    /** */
-    private final ConcurrentLinkedQueue<E> bag = new ConcurrentLinkedQueue<>();
-
-    /** */
-    private final int poolSize;
-
-    /** The function to close object. */
-    private final Consumer<E> closer;
-
-    /** The listener is called when object is returned to the pool. */
-    private final Consumer<E> recycler;
-
-    /**
-     * @param objectFactory Factory used for new objects creation.
-     * @param poolSize Number of objects which pool can contain.
-     * @param closer Function to close object.
-     * @param recycler The listener is called when object is returned to the pool.
-     */
-    public ObjectPool(Supplier<E> objectFactory, int poolSize, Consumer<E> closer, Consumer<E> recycler) {
-        this.objectFactory = objectFactory;
-        this.poolSize = poolSize;
-        this.closer = closer != null ? closer : U::closeQuiet;
-        this.recycler = recycler;
-    }
-
-    /**
-     * Picks an object from the pool if one is present or creates new one otherwise.
-     * Returns an object wrapper which could be returned to the pool.
-     *
-     * @return Reusable object wrapper.
-     */
-    public ObjectPoolReusable<E> borrow() {
-        E pooled = bag.poll();
-
-        return new ObjectPoolReusable<>(this, pooled != null ? pooled : objectFactory.get());
-    }
-
-    /**
-     * Recycles an object.
-     *
-     * @param object Object.
-     */
-    void recycle(E object) {
-        assert object != null  : "Already recycled";
-
-        if (bag.size() < poolSize) {
-            bag.add(object);
-
-            if (recycler != null)
-                recycler.accept(object);
-        }
-        else
-            closer.accept(object);
-    }
-
-    /**
-     * Visible for test
-     * @return Pool bag size.
-     */
-    int bagSize() {
-        return bag.size();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolReusable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolReusable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolReusable.java
deleted file mode 100644
index 48fee42..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolReusable.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.query.h2;
-
-/**
- * Wrapper for a pooled object with capability to return the object to a pool.
- *
- * @param <T> Enclosed object type.
- */
-public class ObjectPoolReusable<T extends AutoCloseable> {
-    /** Object pool to recycle. */
-    private final ObjectPool<T> pool;
-
-    /** Detached object. */
-    private T object;
-
-    /**
-     * @param pool Object pool.
-     * @param object Detached object.
-     */
-    ObjectPoolReusable(ObjectPool<T> pool, T object) {
-        this.pool = pool;
-        this.object = object;
-    }
-
-    /**
-     * @return Enclosed object.
-     */
-    public T object() {
-        return object;
-    }
-
-    /**
-     * Returns an object to a pool or closes it if the pool is already full.
-     */
-    public void recycle() {
-        assert object != null  : "Already recycled";
-
-        pool.recycle(object);
-
-        object = null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java
new file mode 100644
index 0000000..25daa23
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java
@@ -0,0 +1,103 @@
+/*
+ * 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.query.h2;
+
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Special pool for managing limited number objects for further reuse.
+ * This pool maintains separate object bag for each thread by means of {@link ThreadLocal}.
+ * <p>
+ * If object is borrowed on one thread and recycled on different then it will be returned to
+ * recycling thread bag. For thread-safe use either pooled objects should be thread-safe or
+ * <i>happens-before</i> should be established between borrowing object and subsequent recycling.
+ *
+ * @param <E> pooled objects type
+ */
+public final class ThreadLocalObjectPool<E extends AutoCloseable> {
+    /**
+     * Wrapper for a pooled object with capability to return the object to a pool.
+     *
+     * @param <T> enclosed object type
+     */
+    public static class Reusable<T extends AutoCloseable> {
+        /** */
+        private final ThreadLocalObjectPool<T> pool;
+        /** */
+        private final T object;
+
+        /** */
+        private Reusable(ThreadLocalObjectPool<T> pool, T object) {
+            this.pool = pool;
+            this.object = object;
+        }
+
+        /**
+         * @return enclosed object
+         */
+        public T object() {
+            return object;
+        }
+
+        /**
+         * Returns an object to a pool or closes it if the pool is already full.
+         */
+        public void recycle() {
+            Queue<Reusable<T>> bag = pool.bag.get();
+            if (bag.size() < pool.poolSize)
+                bag.add(this);
+            else
+                U.closeQuiet(object);
+        }
+    }
+
+    /** */
+    private final Supplier<E> objectFactory;
+    /** */
+    private final ThreadLocal<Queue<Reusable<E>>> bag = ThreadLocal.withInitial(LinkedList::new);
+    /** */
+    private final int poolSize;
+
+    /**
+     * @param objectFactory factory used for new objects creation
+     * @param poolSize number of objects which pool can contain
+     */
+    public ThreadLocalObjectPool(Supplier<E> objectFactory, int poolSize) {
+        this.objectFactory = objectFactory;
+        this.poolSize = poolSize;
+    }
+
+    /**
+     * Picks an object from the pool if one is present or creates new one otherwise.
+     * Returns an object wrapper which could be returned to the pool.
+     *
+     * @return reusable object wrapper
+     */
+    public Reusable<E> borrow() {
+        Reusable<E> pooled = bag.get().poll();
+        return pooled != null ? pooled : new Reusable<>(this, objectFactory.get());
+    }
+
+    /** Visible for test */
+    int bagSize() {
+        return bag.get().size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
index 31a444e..ba4b12b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
@@ -36,7 +36,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.UpdateSourceIterator;
 import org.apache.ignite.internal.processors.query.h2.H2ConnectionWrapper;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
-import org.apache.ignite.internal.processors.query.h2.ObjectPoolReusable;
+import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPool;
 import org.apache.ignite.internal.processors.query.h2.UpdateResult;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
@@ -623,7 +623,7 @@ public final class UpdatePlan {
         private final EnlistOperation op;
 
         /** */
-        private volatile ObjectPoolReusable<H2ConnectionWrapper> conn;
+        private volatile ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> conn;
 
         /**
          * @param idx Indexing.
@@ -647,7 +647,7 @@ public final class UpdatePlan {
 
         /** {@inheritDoc} */
         @Override public void beforeDetach() {
-            ObjectPoolReusable<H2ConnectionWrapper> conn0 = conn = idx.detachConnection();
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> conn0 = conn = idx.detach();
 
             if (isClosed())
                 conn0.recycle();
@@ -657,7 +657,7 @@ public final class UpdatePlan {
         @Override protected void onClose() {
             cur.close();
 
-            ObjectPoolReusable<H2ConnectionWrapper> conn0 = conn;
+            ThreadLocalObjectPool.Reusable<H2ConnectionWrapper> conn0 = conn;
 
             if (conn0 != null)
                 conn0.recycle();

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index 9971b78..f12c0f3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -18,10 +18,8 @@
 package org.apache.ignite.internal.processors.query.h2.opt;
 
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -91,7 +89,7 @@ public class GridH2QueryContext {
     private MvccSnapshot mvccSnapshot;
 
     /** */
-    private Set<GridH2Table> lockedTables = new HashSet<>();
+    private MapQueryLazyWorker lazyWorker;
 
     /**
      * @param locNodeId Local node ID.
@@ -353,8 +351,7 @@ public class GridH2QueryContext {
          assert qctx.get() == null;
 
          // We need MAP query context to be available to other threads to run distributed joins.
-         if (x.key.type == MAP && x.distributedJoinMode() != OFF && qctxs.putIfAbsent(x.key, x) != null
-             && MapQueryLazyWorker.currentWorker() == null)
+         if (x.key.type == MAP && x.distributedJoinMode() != OFF && qctxs.putIfAbsent(x.key, x) != null)
              throw new IllegalStateException("Query context is already set.");
 
          qctx.set(x);
@@ -404,7 +401,10 @@ public class GridH2QueryContext {
 
         assert x.key.equals(key);
 
-        x.clearContext(nodeStop);
+        if (x.lazyWorker() != null)
+            x.lazyWorker().stop(nodeStop);
+        else
+            x.clearContext(nodeStop);
 
         return true;
     }
@@ -413,10 +413,7 @@ public class GridH2QueryContext {
      * @param nodeStop Node is stopping.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    public synchronized void clearContext(boolean nodeStop) {
-        if (cleared)
-            return;
-
+    public void clearContext(boolean nodeStop) {
         cleared = true;
 
         List<GridReservable> r = reservations;
@@ -519,10 +516,20 @@ public class GridH2QueryContext {
     }
 
     /**
-     * @return The set of tables have been locked by current thread.
+     * @return Lazy worker, if any, or {@code null} if none.
      */
-    public Set<GridH2Table> lockedTables() {
-        return lockedTables;
+    public MapQueryLazyWorker lazyWorker() {
+        return lazyWorker;
+    }
+
+    /**
+     * @param lazyWorker Lazy worker, if any, or {@code null} if none.
+     * @return {@code this}.
+     */
+    public GridH2QueryContext lazyWorker(MapQueryLazyWorker lazyWorker) {
+        this.lazyWorker = lazyWorker;
+
+        return this;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/75e414a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 709ded7..a612b63 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -17,6 +17,17 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -26,7 +37,7 @@ import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryField;
 import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
 import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
-import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker;
+import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.h2.command.ddl.CreateTableData;
 import org.h2.command.dml.Insert;
@@ -47,19 +58,6 @@ import org.h2.table.TableType;
 import org.h2.value.DataType;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.LongAdder;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
@@ -92,12 +90,6 @@ public class GridH2Table extends TableBase {
     /** */
     private final ReadWriteLock lock;
 
-    /** Number of reading threads which currently move execution from query pool to dedicated thread. */
-    private final AtomicInteger lazyTransferCnt = new AtomicInteger();
-
-    /** Has writer that waits lock in the loop. */
-    private volatile boolean hasWaitedWriter;
-
     /** */
     private boolean destroyed;
 
@@ -273,11 +265,6 @@ public class GridH2Table extends TableBase {
 
         ses.addLock(this);
 
-        GridH2QueryContext qctx = GridH2QueryContext.get();
-
-        if (qctx != null)
-           qctx.lockedTables().add(this);
-
         return false;
     }
 
@@ -304,44 +291,15 @@ public class GridH2Table extends TableBase {
         Lock l = exclusive ? lock.writeLock() : lock.readLock();
 
         try {
-            if (exclusive) {
-                // Attempting to obtain exclusive lock for DDL.
-                // Lock is considered acquired only if "lazyTransferCnt" is zero, meaning that 
-                // currently there are no reader threads moving query execution from query 
-                // pool to dedicated thread.
-                // It is possible that reader which is currently transferring execution gets
-                // queued after the write lock we are trying to acquire. So we use timed waiting
-                // and a loop to avoid deadlocks.
-                for (;;) {
-                    if (l.tryLock(200, TimeUnit.MILLISECONDS)) {
-                        if (lazyTransferCnt.get() == 0)
-                            break;
-                        else
-                            l.unlock();
-                    }
-
-                    hasWaitedWriter = true;
-
-                    Thread.yield();
-                }
-
-                hasWaitedWriter = false;
-            }
+            if (!exclusive || !GridMapQueryExecutor.FORCE_LAZY)
+                l.lockInterruptibly();
             else {
-                // Attempt to acquire read lock (query execution, DML, cache update).
-                // If query is being executed inside a query pool, we do not want it to be blocked
-                // for a long time, as it would prevent other queries from being executed. So we
-                // wait a little and then force transfer to dedicated thread by throwing special
-                // timeout exception.GridNioSslSelfTest
-                // If query is not in the query pool, then we simply wait for lock acquisition.
-                if (isSqlNotInLazy()) {
-                    if (hasWaitedWriter || !l.tryLock(200, TimeUnit.MILLISECONDS)) {
-                        throw new GridH2RetryException("Long wait on Table lock: [tableName=" + getName()
-                            + ", hasWaitedWriter=" + hasWaitedWriter + ']');
-                    }
+                for (;;) {
+                    if (l.tryLock(200, TimeUnit.MILLISECONDS))
+                        break;
+                    else
+                        Thread.yield();
                 }
-                else
-                    l.lockInterruptibly();
             }
         }
         catch (InterruptedException e) {
@@ -363,49 +321,6 @@ public class GridH2Table extends TableBase {
     }
 
     /**
-     * Check if table is being locked in not lazy thread by SQL query.
-     *
-     * @return {@code True} if is in query pool.
-     */
-    private static boolean isSqlNotInLazy() {
-        return GridH2QueryContext.get() != null && MapQueryLazyWorker.currentWorker() == null;
-    }
-
-    /**
-     * Callback invoked when session is to be transferred to lazy thread. In order to prevent concurrent changes
-     * by DDL during move we increment counter before releasing read lock.
-     *
-     * @param ses Session.
-     */
-    public void onLazyTransferStarted(Session ses) {
-        assert sessions.containsKey(ses) : "Detached session have not locked the table: " + getName();
-
-        lazyTransferCnt.incrementAndGet();
-
-        lock.readLock().unlock();
-    }
-
-    /**
-     * Callback invoked when lazy transfer finished. Acquire the lock, decrement transfer counter.
-     *
-     * @param ses Session to detach.
-     */
-    public void onLazyTransferFinished(Session ses) {
-        assert sessions.containsKey(ses) : "Attached session have not locked the table: " + getName();
-
-        try {
-            lock.readLock().lockInterruptibly();
-
-            lazyTransferCnt.decrementAndGet();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedException("Thread got interrupted while trying to acquire table lock.", e);
-        }
-    }
-
-    /**
      * Check if table is not destroyed.
      */
     private void ensureNotDestroyed() {
@@ -495,11 +410,6 @@ public class GridH2Table extends TableBase {
         if (exclusive == null)
             return;
 
-        GridH2QueryContext qctx = GridH2QueryContext.get();
-
-        if (qctx != null)
-            qctx.lockedTables().remove(this);
-
         unlock(exclusive);
     }
 
@@ -1039,10 +949,9 @@ public class GridH2Table extends TableBase {
     }
 
     /**
-     * Drop columns.
      *
-     * @param cols Columns.
-     * @param ifExists IF EXISTS flag.
+     * @param cols
+     * @param ifExists
      */
     public void dropColumns(List<String> cols, boolean ifExists) {
         assert !ifExists || cols.size() == 1;


[20/50] [abbrv] ignite git commit: IGNITE-9784 Automatically adjust WAL history size - Fixes #4924.

Posted by sb...@apache.org.
IGNITE-9784 Automatically adjust WAL history size - Fixes #4924.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/82d2efe1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/82d2efe1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/82d2efe1

Branch: refs/heads/ignite-gg-14206
Commit: 82d2efe1a20d148fa2c591265988a1852f68f120
Parents: 7deec60
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Oct 19 12:30:48 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 19 12:30:48 2018 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  4 +-
 .../GridCacheDatabaseSharedManager.java         | 34 +-------
 .../wal/FileWriteAheadLogManager.java           |  9 +-
 .../wal/FsyncModeFileWriteAheadLogManager.java  |  9 +-
 .../ignite/internal/util/IgniteUtils.java       | 86 ++++++++++++++++++++
 5 files changed, 99 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/82d2efe1/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
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 907720a..30d169f 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
@@ -1537,13 +1537,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             for (DataRegionConfiguration dataReg : dataRegions) {
                 res += dataReg.getMaxSize();
 
-                res += GridCacheDatabaseSharedManager.checkpointBufferSize(dataReg);
+                res += U.checkpointBufferSize(dataReg);
             }
         }
 
         res += memCfg.getDefaultDataRegionConfiguration().getMaxSize();
 
-        res += GridCacheDatabaseSharedManager.checkpointBufferSize(memCfg.getDefaultDataRegionConfiguration());
+        res += U.checkpointBufferSize(memCfg.getDefaultDataRegionConfiguration());
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/82d2efe1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 33f593d..73d65f4 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
@@ -172,6 +172,7 @@ 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.pagemem.wal.record.WALRecord.RecordType.CHECKPOINT_RECORD;
 import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID;
+import static org.apache.ignite.internal.util.IgniteUtils.checkpointBufferSize;
 
 /**
  *
@@ -184,15 +185,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** MemoryPolicyConfiguration name reserved for meta store. */
     public static final String METASTORE_DATA_REGION_NAME = "metastoreMemPlc";
 
-    /** */
-    private static final long GB = 1024L * 1024 * 1024;
-
-    /** Minimum checkpointing page buffer size (may be adjusted by Ignite). */
-    public static final Long DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE = GB / 4;
-
-    /** Default minimum checkpointing page buffer size (may be adjusted by Ignite). */
-    public static final Long DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE = 2 * GB;
-
     /** Skip sync. */
     private final boolean skipSync = IgniteSystemProperties.getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC);
 
@@ -680,30 +672,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
     }
 
-    /**
-     * Get checkpoint buffer size for the given configuration.
-     *
-     * @param regCfg Configuration.
-     * @return Checkpoint buffer size.
-     */
-    public static long checkpointBufferSize(DataRegionConfiguration regCfg) {
-        if (!regCfg.isPersistenceEnabled())
-            return 0L;
-
-        long res = regCfg.getCheckpointPageBufferSize();
-
-        if (res == 0L) {
-            if (regCfg.getMaxSize() < GB)
-                res = Math.min(DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE, regCfg.getMaxSize());
-            else if (regCfg.getMaxSize() < 8 * GB)
-                res = regCfg.getMaxSize() / 4;
-            else
-                res = DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE;
-        }
-
-        return res;
-    }
-
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext ctx) throws IgniteCheckedException {
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/82d2efe1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 8198149..0b03128 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -276,7 +276,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      * Maximum number of allowed segments without checkpoint. If we have their more checkpoint should be triggered.
      * It is simple way to calculate WAL size without checkpoint instead fair WAL size calculating.
      */
-    private final long maxSegCountWithoutCheckpoint;
+    private long maxSegCountWithoutCheckpoint;
 
     /** Size of wal archive since which removing of old archive should be started */
     private final long allowedThresholdWalArchiveSize;
@@ -417,9 +417,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         segmentFileInputFactory = new SimpleSegmentFileInputFactory();
         walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity();
 
-        maxSegCountWithoutCheckpoint =
-            (long)((dsCfg.getMaxWalArchiveSize() * CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE) / dsCfg.getWalSegmentSize());
-
         allowedThresholdWalArchiveSize = (long)(dsCfg.getMaxWalArchiveSize() * THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE);
 
         evt = ctx.event();
@@ -438,6 +435,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         if (!cctx.kernalContext().clientNode()) {
+            maxSegCountWithoutCheckpoint =
+                (long)((U.adjustedWalHistorySize(dsCfg, log) * CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE)
+                    / dsCfg.getWalSegmentSize());
+
             final PdsFolderSettings resolveFolders = cctx.kernalContext().pdsFolderResolver().resolveFolders();
 
             checkWalConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/82d2efe1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
index 0ede897..d1e0ebc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java
@@ -218,7 +218,7 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
      * Maximum number of allowed segments without checkpoint. If we have their more checkpoint should be triggered.
      * It is simple way to calculate wal size without checkpoint instead fair wal size calculating.
      */
-    private final long maxSegCountWithoutCheckpoint;
+    private long maxSegCountWithoutCheckpoint;
 
     /** Size of wal archive since which removing of old archive should be started */
     private final long allowedThresholdWalArchiveSize;
@@ -355,9 +355,6 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
         walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity();
         evt = ctx.event();
 
-        maxSegCountWithoutCheckpoint =
-            (long)((dsCfg.getMaxWalArchiveSize() * CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE) / dsCfg.getWalSegmentSize());
-
         allowedThresholdWalArchiveSize = (long)(dsCfg.getMaxWalArchiveSize() * THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE);
 
         assert mode == WALMode.FSYNC : dsCfg;
@@ -375,6 +372,10 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         if (!cctx.kernalContext().clientNode()) {
+            maxSegCountWithoutCheckpoint =
+                (long)((U.adjustedWalHistorySize(dsCfg, log) * CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE)
+                    / dsCfg.getWalSegmentSize());
+
             final PdsFolderSettings resolveFolders = cctx.kernalContext().pdsFolderResolver().resolveFolders();
 
             checkWalConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/82d2efe1/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 d2f45c9..d10a7c7 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
@@ -180,6 +180,8 @@ import org.apache.ignite.compute.ComputeTaskCancelledException;
 import org.apache.ignite.compute.ComputeTaskName;
 import org.apache.ignite.compute.ComputeTaskTimeoutException;
 import org.apache.ignite.configuration.AddressResolver;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
@@ -281,6 +283,15 @@ import static org.apache.ignite.internal.util.GridUnsafe.staticFieldOffset;
  */
 @SuppressWarnings({"UnusedReturnValue", "UnnecessaryFullyQualifiedName", "RedundantStringConstructorCall"})
 public abstract class IgniteUtils {
+    /** */
+    private static final long GB = 1024L * 1024 * 1024;
+
+    /** Minimum checkpointing page buffer size (may be adjusted by Ignite). */
+    public static final Long DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE = GB / 4;
+
+    /** Default minimum checkpointing page buffer size (may be adjusted by Ignite). */
+    public static final Long DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE = 2 * GB;
+
     /** {@code True} if {@code unsafe} should be used for array copy. */
     private static final boolean UNSAFE_BYTE_ARR_CP = unsafeByteArrayCopyAvailable();
 
@@ -10399,6 +10410,81 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Get checkpoint buffer size for the given configuration.
+     *
+     * @param regCfg Configuration.
+     * @return Checkpoint buffer size.
+     */
+    public static long checkpointBufferSize(DataRegionConfiguration regCfg) {
+        if (!regCfg.isPersistenceEnabled())
+            return 0L;
+
+        long res = regCfg.getCheckpointPageBufferSize();
+
+        if (res == 0L) {
+            if (regCfg.getMaxSize() < GB)
+                res = Math.min(DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE, regCfg.getMaxSize());
+            else if (regCfg.getMaxSize() < 8 * GB)
+                res = regCfg.getMaxSize() / 4;
+            else
+                res = DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE;
+        }
+
+        return res;
+    }
+
+    /**
+     * Calculates maximum WAL archive size based on maximum checkpoint buffer size, if the default value of
+     * {@link DataStorageConfiguration#getMaxWalArchiveSize()} is not overridden.
+     *
+     * @return User-set max WAL archive size of triple size of the maximum checkpoint buffer.
+     */
+    public static long adjustedWalHistorySize(DataStorageConfiguration dsCfg, @Nullable IgniteLogger log) {
+        if (dsCfg.getMaxWalArchiveSize() != DataStorageConfiguration.DFLT_WAL_ARCHIVE_MAX_SIZE)
+            return dsCfg.getMaxWalArchiveSize();
+
+        // Find out the maximum checkpoint buffer size.
+        long maxCpBufSize = 0;
+
+        if (dsCfg.getDataRegionConfigurations() != null) {
+            for (DataRegionConfiguration regCfg : dsCfg.getDataRegionConfigurations()) {
+                long cpBufSize = checkpointBufferSize(regCfg);
+
+                if (cpBufSize > regCfg.getMaxSize())
+                    cpBufSize = regCfg.getMaxSize();
+
+                if (cpBufSize > maxCpBufSize)
+                    maxCpBufSize = cpBufSize;
+            }
+        }
+
+        {
+            DataRegionConfiguration regCfg = dsCfg.getDefaultDataRegionConfiguration();
+
+            long cpBufSize = checkpointBufferSize(regCfg);
+
+            if (cpBufSize > regCfg.getMaxSize())
+                cpBufSize = regCfg.getMaxSize();
+
+            if (cpBufSize > maxCpBufSize)
+                maxCpBufSize = cpBufSize;
+        }
+
+        long adjustedWalArchiveSize = maxCpBufSize * 4;
+
+        if (adjustedWalArchiveSize > dsCfg.getMaxWalArchiveSize()) {
+            if (log != null)
+                U.quietAndInfo(log, "Automatically adjusted max WAL archive size to " +
+                    U.readableSize(adjustedWalArchiveSize, false) +
+                    " (to override, use DataStorageConfiguration.setMaxWalArhiveSize)");
+
+            return adjustedWalArchiveSize;
+        }
+
+        return dsCfg.getMaxWalArchiveSize();
+    }
+
+    /**
      * Return count of regular file in the directory (including in sub-directories)
      *
      * @param dir path to directory


[31/50] [abbrv] ignite git commit: IGNITE-9945 Entry start version should not omit datacenter ID - Fixes #5034.

Posted by sb...@apache.org.
IGNITE-9945 Entry start version should not omit datacenter ID - Fixes #5034.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4353618a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4353618a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4353618a

Branch: refs/heads/ignite-gg-14206
Commit: 4353618a7501e2dada6b0027cadc2b9c6b069fde
Parents: ebaedf9
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Mon Oct 22 16:01:28 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 22 16:01:28 2018 +0300

----------------------------------------------------------------------
 .../cache/CacheConflictResolutionManager.java   |  2 +-
 .../processors/cache/GridCacheMapEntry.java     |  9 ++----
 .../cache/version/GridCacheVersionManager.java  | 31 ++++++++++++++++++--
 3 files changed, 32 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4353618a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConflictResolutionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConflictResolutionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConflictResolutionManager.java
index 6d65d82..9790f75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConflictResolutionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheConflictResolutionManager.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.internal.processors.cache.version.*;
+import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver;
 
 /**
  * Conflict resolver manager.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4353618a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
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 ab5b725..c753296 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
@@ -73,7 +73,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVer
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
@@ -117,10 +116,8 @@ import static org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicR
 import static org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome.REMOVE_NO_VAL;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.compareIgnoreOpCounter;
 import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.NO_KEY;
-import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.CONCURRENT_UPDATE;
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.DUPLICATE_KEY;
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.TRANSACTION_SERIALIZATION_ERROR;
-import static org.apache.ignite.internal.processors.dr.GridDrType.DR_BACKUP;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
 
 /**
@@ -222,7 +219,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         this.cctx = cctx;
         this.listenerLock = cctx.continuousQueries().getListenerReadLock();
 
-        ver = GridCacheVersionManager.START_VER;
+        ver = cctx.shared().versions().startVersion();
     }
 
     /**
@@ -332,7 +329,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @return {@code True} if start version.
      */
     public boolean isStartVersion() {
-        return ver == GridCacheVersionManager.START_VER;
+        return cctx.shared().versions().isStartVersion(ver);
     }
 
     /** {@inheritDoc} */
@@ -3287,7 +3284,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                     GridCacheVersion currentVer = row != null ? row.version() : GridCacheMapEntry.this.ver;
 
-                    boolean isStartVer = currentVer == GridCacheVersionManager.START_VER;
+                    boolean isStartVer = cctx.shared().versions().isStartVersion(currentVer);
 
                     if (cctx.group().persistenceEnabled()) {
                         if (!isStartVer) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4353618a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
index df8af48..2a6526a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
@@ -41,9 +41,6 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     /** */
     public static final GridCacheVersion EVICT_VER = new GridCacheVersion(Integer.MAX_VALUE, 0, 0, 0);
 
-    /** */
-    public static final GridCacheVersion START_VER = new GridCacheVersion(0, 0, 0, 0);
-
     /** Timestamp used as base time for cache topology version (January 1, 2014). */
     public static final long TOP_VER_BASE_TIME = 1388520000000L;
 
@@ -56,6 +53,9 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     /** Current order for store operations. */
     private final AtomicLong loadOrder = new AtomicLong(0);
 
+    /** Entry start version. */
+    private GridCacheVersion startVer;
+
     /** Last version. */
     private volatile GridCacheVersion last;
 
@@ -87,6 +87,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     @Override public void start0() throws IgniteCheckedException {
         last = new GridCacheVersion(0, order.get(), 0, dataCenterId);
 
+        startVer = new GridCacheVersion(0, 0, 0, dataCenterId);
+
         cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_METRICS_UPDATED);
     }
 
@@ -104,6 +106,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
         this.dataCenterId = dataCenterId;
 
         last = new GridCacheVersion(0, order.get(), 0, dataCenterId);
+
+        startVer = new GridCacheVersion(0, 0, 0, dataCenterId);
     }
 
     /**
@@ -301,4 +305,25 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     public GridCacheVersion last() {
         return last;
     }
+
+    /**
+     * Gets start version.
+     *
+     * @return Start version.
+     */
+    public GridCacheVersion startVersion() {
+        assert startVer != null;
+
+        return startVer;
+    }
+
+    /**
+     * Check if given version is start version.
+     *
+     * @param ver Version.
+     * @return {@code True} if given version is start version.
+     */
+    public boolean isStartVersion(GridCacheVersion ver) {
+        return startVer.equals(ver);
+    }
 }


[02/50] [abbrv] ignite git commit: IGNITE-9882: Hadoop: fixed OOME on TC. This closes #5003.

Posted by sb...@apache.org.
IGNITE-9882: Hadoop: fixed OOME on TC. This closes #5003.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/cb0d1de3
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/cb0d1de3
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/cb0d1de3

Branch: refs/heads/ignite-gg-14206
Commit: cb0d1de30c44fb333f5ea7c4db087015a7cfe8b5
Parents: 8828c1d
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Wed Oct 17 17:11:09 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Oct 17 17:11:09 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  4 ++
 .../processors/hadoop/HadoopHelper.java         |  5 ++
 .../processors/hadoop/HadoopNoopHelper.java     |  5 ++
 .../internal/processors/igfs/IgfsImpl.java      |  3 ++
 .../processors/hadoop/HadoopHelperImpl.java     |  8 ++-
 .../impl/HadoopAbstractMapReduceTest.java       | 18 +++++++
 .../taskexecutor/HadoopExecutorServiceTest.java | 52 ++------------------
 7 files changed, 45 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 7f4310f..c6ec9be 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
@@ -2289,6 +2289,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 }
             }
 
+            if (ctx.hadoopHelper() != null)
+                ctx.hadoopHelper().close();
+
             if (starveTask != null)
                 starveTask.close();
 
@@ -2377,6 +2380,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             MarshallerExclusions.clearCache();
             BinaryEnumCache.clear();
 
+
             gw.writeLock();
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java
index 7936fef..ae9985d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java
@@ -59,4 +59,9 @@ public interface HadoopHelper {
      * @return Work directory.
      */
     public String workDirectory();
+
+    /**
+     * Close helper.
+     */
+    public void close();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java
index f8f870f..986af1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java
@@ -62,6 +62,11 @@ public class HadoopNoopHelper implements HadoopHelper {
         throw unsupported();
     }
 
+    /** {@inheritDoc} */
+    @Override public void close() {
+        // No-op.
+    }
+
     /**
      * @return Exception.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index ff53e00..bac536d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -276,6 +276,9 @@ public final class IgfsImpl implements IgfsEx {
         // Restore interrupted flag.
         if (interrupted)
             Thread.currentThread().interrupt();
+
+        if (dualPool != null)
+            dualPool.shutdownNow();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java
index 0e86529..6da79b2 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java
@@ -35,7 +35,7 @@ import java.io.InputStream;
  */
 public class HadoopHelperImpl implements HadoopHelper {
     /** Kernal context. */
-    private final GridKernalContext ctx;
+    private GridKernalContext ctx;
 
     /** Common class loader. */
     private volatile HadoopClassLoader ldr;
@@ -130,4 +130,10 @@ public class HadoopHelperImpl implements HadoopHelper {
             throw new IgniteException("Failed to resolve Ignite work directory.", e);
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        // Force drop KernalContext link, because HadoopHelper leaks in some tests.
+        ctx = null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java
index fc6d7f8..fa224b2 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.UUID;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -55,6 +56,7 @@ import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2;
+import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
@@ -344,6 +346,22 @@ public class HadoopAbstractMapReduceTest extends HadoopAbstractWordCountTest {
         super.beforeTest();
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        igniteSecondary = null;
+        secondaryFs = null;
+        igfs = null;
+
+        HadoopFileSystemsUtils.clearFileSystemCache();
+        FileSystem.clearStatistics();
+
+        Map stat = GridTestUtils.getFieldValue(FileSystem.class, FileSystem.class, "statisticsTable");
+
+        stat.clear();
+    }
+
     /**
      * Start grid with IGFS.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb0d1de3/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java
index 3486a14..cc6cbb3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java
@@ -17,14 +17,13 @@
 
 package org.apache.ignite.internal.processors.hadoop.impl.taskexecutor;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.LongAdder;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopExecutorService;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.LongAdder;
+
 /**
  *
  */
@@ -70,49 +69,4 @@ public class HadoopExecutorServiceTest extends GridCommonAbstractTest {
 
         assertTrue(exec.shutdown(0));
     }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShutdown() throws Exception {
-        for (int i = 0; i < 5; i++) {
-            final HadoopExecutorService exec = new HadoopExecutorService(log, "_GRID_NAME_", 10, 5);
-
-            final LongAdder sum = new LongAdder();
-
-            final AtomicBoolean finish = new AtomicBoolean();
-
-            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    while (!finish.get()) {
-                        exec.submit(new Callable<Void>() {
-                            @Override public Void call() throws Exception {
-                                sum.increment();
-
-                                return null;
-                            }
-                        });
-                    }
-
-                    return null;
-                }
-            }, 19);
-
-            Thread.sleep(200);
-
-            assertTrue(exec.shutdown(50));
-
-            long res = sum.sum();
-
-            assertTrue(res > 0);
-
-            finish.set(true);
-
-            fut.get();
-
-            assertEquals(res, sum.sum()); // Nothing was executed after shutdown.
-
-            X.println("_ ok");
-        }
-    }
 }
\ No newline at end of file


[13/50] [abbrv] ignite git commit: IGNITE-9082 Throwing checked exception during tx commit without node stopping leads to data corruption - Fixes #4809.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java
new file mode 100644
index 0000000..881c680
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java
@@ -0,0 +1,234 @@
+/*
+ * 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.cache.transactions;
+
+import java.util.UUID;
+import java.util.function.Supplier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testsuites.IgniteIgnore;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Tests data consistency if transaction is failed due to heuristic exception on originating node.
+ */
+public class TxDataConsistencyOnCommitFailureTest extends GridCommonAbstractTest {
+    /** */
+    public static final int KEY = 0;
+
+    /** */
+    public static final String CLIENT = "client";
+
+    /** */
+    private int nodesCnt;
+
+    /** */
+    private int backups;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setClientMode(igniteInstanceName.startsWith(CLIENT));
+
+        cfg.setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME).
+            setCacheMode(CacheMode.PARTITIONED).
+            setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL).
+            setBackups(backups).
+            setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** */
+    public void testCommitErrorOnNearNode2PC() throws Exception {
+        nodesCnt = 3;
+
+        backups = 2;
+
+        doTestCommitError(() -> {
+            try {
+                return startGrid("client");
+            }
+            catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        });
+    }
+
+    /** */
+    public void testCommitErrorOnNearNode1PC() throws Exception {
+        nodesCnt = 2;
+
+        backups = 1;
+
+        doTestCommitError(() -> {
+            try {
+                return startGrid("client");
+            }
+            catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        });
+    }
+
+    /** */
+    @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-9806", forceFailure = false)
+    public void testCommitErrorOnColocatedNode2PC() throws Exception {
+        nodesCnt = 3;
+
+        backups = 2;
+
+        doTestCommitError(() -> primaryNode(KEY, DEFAULT_CACHE_NAME));
+    }
+
+    /**
+     * @param factory Factory.
+     */
+    private void doTestCommitError(Supplier<Ignite> factory) throws Exception {
+        Ignite crd = startGridsMultiThreaded(nodesCnt);
+
+        crd.cache(DEFAULT_CACHE_NAME).put(KEY, KEY);
+
+        Ignite ignite = factory.get();
+
+        if (ignite == null)
+            ignite = startGrid("client");
+
+        assertNotNull(ignite.cache(DEFAULT_CACHE_NAME));
+
+        injectMockedTxManager(ignite);
+
+        checkKey();
+
+        IgniteTransactions transactions = ignite.transactions();
+
+        try(Transaction tx = transactions.txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ, 0, 1)) {
+            assertNotNull(transactions.tx());
+
+            ignite.cache(DEFAULT_CACHE_NAME).put(KEY, KEY + 1);
+
+            tx.commit();
+
+            fail();
+        }
+        catch (Exception t) {
+            // No-op.
+        }
+
+        checkKey();
+
+        checkFutures();
+    }
+
+    /**
+     * @param ignite Ignite.
+     */
+    private void injectMockedTxManager(Ignite ignite) {
+        IgniteEx igniteEx = (IgniteEx)ignite;
+
+        GridCacheSharedContext<Object, Object> ctx = igniteEx.context().cache().context();
+
+        IgniteTxManager tm = ctx.tm();
+
+        IgniteTxManager mockTm = Mockito.spy(tm);
+
+        MockGridNearTxLocal locTx = new MockGridNearTxLocal(ctx, false, false, false, GridIoPolicy.SYSTEM_POOL,
+            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ, 0, true, null, 1, null, 0, null);
+
+        Mockito.doAnswer(new Answer<GridNearTxLocal>() {
+            @Override public GridNearTxLocal answer(InvocationOnMock invocation) throws Throwable {
+                mockTm.onCreated(null, locTx);
+
+                return locTx;
+            }
+        }).when(mockTm).
+            newTx(locTx.implicit(), locTx.implicitSingle(), null, locTx.concurrency(),
+                locTx.isolation(), locTx.timeout(), locTx.storeEnabled(), null, locTx.size(), locTx.label());
+
+        ctx.setTxManager(mockTm);
+    }
+
+    /** */
+    private void checkKey() {
+        for (Ignite ignite : G.allGrids()) {
+            if (!ignite.configuration().isClientMode())
+                assertNotNull(ignite.cache(DEFAULT_CACHE_NAME).localPeek(KEY));
+        }
+    }
+
+    /** */
+    private static class MockGridNearTxLocal extends GridNearTxLocal {
+        /** Empty constructor. */
+        public MockGridNearTxLocal() {
+        }
+
+        /**
+         * @param ctx Context.
+         * @param implicit Implicit.
+         * @param implicitSingle Implicit single.
+         * @param sys System.
+         * @param plc Policy.
+         * @param concurrency Concurrency.
+         * @param isolation Isolation.
+         * @param timeout Timeout.
+         * @param storeEnabled Store enabled.
+         * @param mvccOp Mvcc op.
+         * @param txSize Tx size.
+         * @param subjId Subj id.
+         * @param taskNameHash Task name hash.
+         * @param lb Label.
+         */
+        public MockGridNearTxLocal(GridCacheSharedContext ctx, boolean implicit, boolean implicitSingle, boolean sys,
+            byte plc, TransactionConcurrency concurrency, TransactionIsolation isolation, long timeout,
+            boolean storeEnabled, Boolean mvccOp, int txSize, @Nullable UUID subjId, int taskNameHash, @Nullable String lb) {
+            super(ctx, implicit, implicitSingle, sys, plc, concurrency, isolation, timeout, storeEnabled, mvccOp,
+                txSize, subjId, taskNameHash, lb);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void userCommit() throws IgniteCheckedException {
+            throw new IgniteCheckedException("Force failure");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 7e98ec7..7091a09 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -1996,17 +1996,26 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
             final Collection<GridCacheFuture<?>> futs = ig.context().cache().context().mvcc().activeFutures();
 
-            for (GridCacheFuture<?> fut : futs)
-                log.info("Waiting for future: " + fut);
+            boolean hasFutures = false;
 
-            assertTrue("Expecting no active futures: node=" + ig.localNode().id(), futs.isEmpty());
+            for (GridCacheFuture<?> fut : futs) {
+                if (!fut.isDone()) {
+                    log.error("Expecting no active future [node=" + ig.localNode().id() + ", fut=" + fut + ']');
+
+                    hasFutures = true;
+                }
+            }
+
+            if (hasFutures)
+                fail("Some mvcc futures are not finished");
 
             Collection<IgniteInternalTx> txs = ig.context().cache().context().tm().activeTransactions();
 
             for (IgniteInternalTx tx : txs)
-                log.info("Waiting for tx: " + tx);
+                log.error("Expecting no active transaction [node=" + ig.localNode().id() + ", tx=" + tx + ']');
 
-            assertTrue("Expecting no active transactions: node=" + ig.localNode().id(), txs.isEmpty());
+            if (!txs.isEmpty())
+                fail("Some transaction are not finished");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5eb871e1/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
index 386b17b..7dba461 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicPrimar
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCachePrimarySyncTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxCachePrimarySyncTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxCacheWriteSynchronizationModesMultithreadedTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxDataConsistencyOnCommitFailureTest;
 import org.apache.ignite.testframework.junits.GridAbstractTest;
 
 /**
@@ -49,6 +50,8 @@ public class IgniteCacheTestSuite9 extends TestSuite {
 
         suite.addTestSuite(CacheAtomicPrimarySyncBackPressureTest.class);
 
+        suite.addTestSuite(TxDataConsistencyOnCommitFailureTest.class);
+
         return suite;
     }
 }


[42/50] [abbrv] ignite git commit: IGNITE-9719 Extra rebalanceThreadPoolSize check on client node - Fixes #4911.

Posted by sb...@apache.org.
IGNITE-9719 Extra rebalanceThreadPoolSize check on client node - Fixes #4911.

Signed-off-by: Ivan Rakov <ir...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/962d6a29
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/962d6a29
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/962d6a29

Branch: refs/heads/ignite-gg-14206
Commit: 962d6a29716195c13519ca48b6d79f3b8541653c
Parents: d4af213
Author: luchnikovnsk <lu...@gmail.com>
Authored: Tue Oct 23 11:16:40 2018 +0300
Committer: Ivan Rakov <ir...@apache.org>
Committed: Tue Oct 23 11:16:40 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 32 +++++++++------
 .../ignite/client/ClientConfigurationTest.java  | 42 ++++++++++++++++++++
 2 files changed, 61 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/962d6a29/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 40347d7..1546da3 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
@@ -2581,19 +2581,25 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      *
      */
     private void ackRebalanceConfiguration() throws IgniteCheckedException {
-        if (cfg.getSystemThreadPoolSize() <= cfg.getRebalanceThreadPoolSize())
-            throw new IgniteCheckedException("Rebalance thread pool size exceed or equals System thread pool size. " +
-                "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
-
-        if (cfg.getRebalanceThreadPoolSize() < 1)
-            throw new IgniteCheckedException("Rebalance thread pool size minimal allowed value is 1. " +
-                "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
-
-        for (CacheConfiguration ccfg : cfg.getCacheConfiguration()) {
-            if (ccfg.getRebalanceBatchesPrefetchCount() < 1)
-                throw new IgniteCheckedException("Rebalance batches prefetch count minimal allowed value is 1. " +
-                    "Change CacheConfiguration.rebalanceBatchesPrefetchCount property before next start. " +
-                    "[cache=" + ccfg.getName() + "]");
+        if (cfg.isClientMode()) {
+            if (cfg.getRebalanceThreadPoolSize() != IgniteConfiguration.DFLT_REBALANCE_THREAD_POOL_SIZE)
+                U.warn(log, "Setting the rebalance pool size has no effect on the client mode");
+        }
+        else {
+            if (cfg.getSystemThreadPoolSize() <= cfg.getRebalanceThreadPoolSize())
+                throw new IgniteCheckedException("Rebalance thread pool size exceed or equals System thread pool size. " +
+                    "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
+
+            if (cfg.getRebalanceThreadPoolSize() < 1)
+                throw new IgniteCheckedException("Rebalance thread pool size minimal allowed value is 1. " +
+                    "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
+
+            for (CacheConfiguration ccfg : cfg.getCacheConfiguration()) {
+                if (ccfg.getRebalanceBatchesPrefetchCount() < 1)
+                    throw new IgniteCheckedException("Rebalance batches prefetch count minimal allowed value is 1. " +
+                        "Change CacheConfiguration.rebalanceBatchesPrefetchCount property before next start. " +
+                        "[cache=" + ccfg.getName() + "]");
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/962d6a29/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
index bcc212a..287c6ec 100644
--- a/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
@@ -25,9 +25,19 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutput;
 import java.io.ObjectOutputStream;
 import java.util.Collections;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridStringLogger;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -73,4 +83,36 @@ public class ClientConfigurationTest {
 
         assertTrue(Comparers.equal(target, desTarget));
     }
+
+    /**
+     * Test check the case when {@link IgniteConfiguration#getRebalanceThreadPoolSize()} is equal to {@link
+     * IgniteConfiguration#getSystemThreadPoolSize()}
+     */
+    @Test
+    public void testRebalanceThreadPoolSize() {
+        GridStringLogger gridStrLog = new GridStringLogger();
+        gridStrLog.logLength(1024 * 100);
+
+        IgniteConfiguration cci = Config.getServerConfiguration().setClientMode(true);
+        cci.setRebalanceThreadPoolSize(cci.getSystemThreadPoolSize());
+        cci.setGridLogger(gridStrLog);
+
+        try (
+            Ignite si = Ignition.start(Config.getServerConfiguration());
+            Ignite ci = Ignition.start(cci)) {
+            Set<ClusterNode> collect = si.cluster().nodes().stream()
+                .filter(new Predicate<ClusterNode>() {
+                    @Override public boolean test(ClusterNode clusterNode) {
+                        return clusterNode.isClient();
+                    }
+                })
+                .collect(Collectors.toSet());
+
+            String log = gridStrLog.toString();
+            boolean containsMsg = log.contains("Setting the rebalance pool size has no effect on the client mode");
+
+            Assert.assertTrue(containsMsg);
+            Assert.assertEquals(1, collect.size());
+        }
+    }
 }


[23/50] [abbrv] ignite git commit: IGNITE-5795 Register binary metadata during cache start - Fixes #4852.

Posted by sb...@apache.org.
IGNITE-5795 Register binary metadata during cache start - Fixes #4852.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3bb03444
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3bb03444
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3bb03444

Branch: refs/heads/ignite-gg-14206
Commit: 3bb03444246f863096063d084393676a84d2bc0e
Parents: 5939a94
Author: Anton Kalashnikov <ka...@yandex.ru>
Authored: Fri Oct 19 17:51:41 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 19 17:56:36 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../binary/BinaryCachingMetadataHandler.java    |  25 +-
 .../ignite/internal/binary/BinaryContext.java   |  59 +++-
 .../internal/binary/BinaryMetadataHandler.java  |  10 +
 .../binary/BinaryNoopMetadataHandler.java       |   6 +
 .../binary/builder/BinaryObjectBuilderImpl.java |   2 +-
 .../internal/client/thin/TcpIgniteClient.java   |   6 +
 .../processors/cache/GridCacheProcessor.java    |   9 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   5 +
 .../processors/query/GridQueryProcessor.java    |  77 ++++-
 .../binary/TestCachingMetadataHandler.java      |   6 +
 .../cache/CacheRegisterMetadataLocallyTest.java | 287 +++++++++++++++++++
 .../cache/index/AbstractSchemaSelfTest.java     |   6 +-
 .../index/H2DynamicIndexAbstractSelfTest.java   |  48 ++--
 .../IgniteCacheWithIndexingTestSuite.java       |   2 +
 15 files changed, 494 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 c6ec9be..40347d7 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
@@ -1005,6 +1005,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Start processors before discovery manager, so they will
             // be able to start receiving messages once discovery completes.
             try {
+                startProcessor(new GridMarshallerMappingProcessor(ctx));
                 startProcessor(new PdsConsistentIdProcessor(ctx));
                 startProcessor(new MvccProcessorImpl(ctx));
                 startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx));
@@ -1028,7 +1029,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 startProcessor(createHadoopComponent());
                 startProcessor(new DataStructuresProcessor(ctx));
                 startProcessor(createComponent(PlatformProcessor.class, ctx));
-                startProcessor(new GridMarshallerMappingProcessor(ctx));
 
                 // Start plugins.
                 for (PluginProvider provider : ctx.plugins().allProviders()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
index a0559cb..b60dc097 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
@@ -46,24 +46,29 @@ public class BinaryCachingMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void addMeta(int typeId, BinaryType type, boolean failIfUnregistered) throws BinaryObjectException {
-        synchronized (this) {
-            BinaryType oldType = metas.put(typeId, type);
+    @Override public synchronized void addMeta(int typeId, BinaryType type,
+        boolean failIfUnregistered) throws BinaryObjectException {
+        BinaryType oldType = metas.put(typeId, type);
 
-            if (oldType != null) {
-                BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
-                BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
+        if (oldType != null) {
+            BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
+            BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
 
-                BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta);
+            BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta);
 
-                BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
+            BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
 
-                metas.put(typeId, mergedType);
-            }
+            metas.put(typeId, mergedType);
         }
     }
 
     /** {@inheritDoc} */
+    @Override public synchronized void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
+        throws BinaryObjectException {
+        addMeta(typeId, meta, failIfUnregistered);
+    }
+
+    /** {@inheritDoc} */
     @Override public synchronized BinaryType metadata(int typeId) throws BinaryObjectException {
         return metas.get(typeId);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 7885d95..7ab74e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -617,6 +617,18 @@ public class BinaryContext {
      */
     public BinaryClassDescriptor descriptorForClass(Class<?> cls, boolean deserialize, boolean failIfUnregistered)
         throws BinaryObjectException {
+        return descriptorForClass(cls, deserialize, failIfUnregistered, false);
+    }
+
+    /**
+     * @param cls Class.
+     * @param failIfUnregistered Throw exception if class isn't registered.
+     * @param onlyLocReg {@code true} if descriptor need to register only locally when registration is required at all.
+     * @return Class descriptor.
+     * @throws BinaryObjectException In case of error.
+     */
+    public BinaryClassDescriptor descriptorForClass(Class<?> cls, boolean deserialize, boolean failIfUnregistered,
+        boolean onlyLocReg) throws BinaryObjectException {
         assert cls != null;
 
         BinaryClassDescriptor desc = descByCls.get(cls);
@@ -625,7 +637,7 @@ public class BinaryContext {
             if (failIfUnregistered)
                 throw new UnregisteredClassException(cls);
 
-            desc = registerClassDescriptor(cls, deserialize);
+            desc = registerClassDescriptor(cls, deserialize, onlyLocReg);
         }
         else if (!desc.registered()) {
             if (!desc.userType()) {
@@ -662,7 +674,7 @@ public class BinaryContext {
                 if (failIfUnregistered)
                     throw new UnregisteredClassException(cls);
 
-                desc = registerUserClassDescriptor(desc);
+                desc = registerUserClassDescriptor(desc, onlyLocReg);
             }
         }
 
@@ -715,7 +727,7 @@ public class BinaryContext {
         }
 
         if (desc == null) {
-            desc = registerClassDescriptor(cls, deserialize);
+            desc = registerClassDescriptor(cls, deserialize, false);
 
             assert desc.typeId() == typeId : "Duplicate typeId [typeId=" + typeId + ", cls=" + cls
                 + ", desc=" + desc + "]";
@@ -728,9 +740,10 @@ public class BinaryContext {
      * Creates and registers {@link BinaryClassDescriptor} for the given {@code class}.
      *
      * @param cls Class.
+     * @param onlyLocReg {@code true} if descriptor need to register only locally when registration is required at all.
      * @return Class descriptor.
      */
-    private BinaryClassDescriptor registerClassDescriptor(Class<?> cls, boolean deserialize) {
+    private BinaryClassDescriptor registerClassDescriptor(Class<?> cls, boolean deserialize, boolean onlyLocReg) {
         BinaryClassDescriptor desc;
 
         String clsName = cls.getName();
@@ -759,7 +772,7 @@ public class BinaryContext {
                 desc = old;
         }
         else
-            desc = registerUserClassDescriptor(cls, deserialize);
+            desc = registerUserClassDescriptor(cls, deserialize, onlyLocReg);
 
         return desc;
     }
@@ -768,9 +781,10 @@ public class BinaryContext {
      * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}.
      *
      * @param cls Class.
+     * @param onlyLocReg {@code true} if descriptor need to register only locally.
      * @return Class descriptor.
      */
-    private BinaryClassDescriptor registerUserClassDescriptor(Class<?> cls, boolean deserialize) {
+    private BinaryClassDescriptor registerUserClassDescriptor(Class<?> cls, boolean deserialize, boolean onlyLocReg) {
         boolean registered;
 
         final String clsName = cls.getName();
@@ -781,7 +795,7 @@ public class BinaryContext {
 
         final int typeId = mapper.typeId(clsName);
 
-        registered = registerUserClassName(typeId, cls.getName());
+        registered = registerUserClassName(typeId, cls.getName(), onlyLocReg);
 
         BinarySerializer serializer = serializerForClass(cls);
 
@@ -799,9 +813,22 @@ public class BinaryContext {
             registered
         );
 
-        if (!deserialize)
-            metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, null,
-                desc.isEnum(), cls.isEnum() ? enumMap(cls) : null).wrap(this), false);
+        if (!deserialize) {
+            BinaryMetadata binaryMetadata = new BinaryMetadata(
+                typeId,
+                typeName,
+                desc.fieldsMeta(),
+                affFieldName,
+                null,
+                desc.isEnum(),
+                cls.isEnum() ? enumMap(cls) : null
+            );
+
+            if (onlyLocReg)
+                metaHnd.addMetaLocally(typeId, binaryMetadata.wrap(this), false);
+            else
+                metaHnd.addMeta(typeId, binaryMetadata.wrap(this), false);
+        }
 
         descByCls.put(cls, desc);
 
@@ -814,12 +841,13 @@ public class BinaryContext {
      * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}.
      *
      * @param desc Old descriptor that should be re-registered.
+     * @param onlyLocReg {@code true} if descriptor need to register only locally.
      * @return Class descriptor.
      */
-    private BinaryClassDescriptor registerUserClassDescriptor(BinaryClassDescriptor desc) {
+    private BinaryClassDescriptor registerUserClassDescriptor(BinaryClassDescriptor desc, boolean onlyLocReg) {
         boolean registered;
 
-        registered = registerUserClassName(desc.typeId(), desc.describedClass().getName());
+        registered = registerUserClassName(desc.typeId(), desc.describedClass().getName(), onlyLocReg);
 
         if (registered) {
             BinarySerializer serializer = desc.initialSerializer();
@@ -1191,15 +1219,18 @@ public class BinaryContext {
      *
      * @param typeId Type ID.
      * @param clsName Class Name.
+     * @param onlyLocReg {@code true} if descriptor need to register only locally.
      * @return {@code True} if the mapping was registered successfully.
      */
-    public boolean registerUserClassName(int typeId, String clsName) {
+    public boolean registerUserClassName(int typeId, String clsName, boolean onlyLocReg) {
         IgniteCheckedException e = null;
 
         boolean res = false;
 
         try {
-            res = marshCtx.registerClassName(JAVA_ID, typeId, clsName);
+            res = onlyLocReg
+                ? marshCtx.registerClassNameLocally(JAVA_ID, typeId, clsName)
+                : marshCtx.registerClassName(JAVA_ID, typeId, clsName);
         }
         catch (DuplicateTypeIdException dupEx) {
             // Ignore if trying to register mapped type name of the already registered class name and vise versa

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
index 85ab137..d1336bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java
@@ -36,6 +36,16 @@ public interface BinaryMetadataHandler {
     public void addMeta(int typeId, BinaryType meta, boolean failIfUnregistered) throws BinaryObjectException;
 
     /**
+     * Adds meta data locally on current node without sending any messages.
+     *
+     * @param typeId Type ID.
+     * @param meta Metadata.
+     * @param failIfUnregistered Fail if unregistered.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered) throws BinaryObjectException;
+
+    /**
      * Gets meta data for provided type ID.
      *
      * @param typeId Type ID.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
index 4ee2428..a552d61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java
@@ -48,6 +48,12 @@ public class BinaryNoopMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
+    @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
+        throws BinaryObjectException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index abd63cd..5414a25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -364,7 +364,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                 if (affFieldName0 == null)
                     affFieldName0 = ctx.affinityKeyFieldName(typeId);
 
-                ctx.registerUserClassName(typeId, typeName);
+                ctx.registerUserClassName(typeId, typeName, false);
 
                 ctx.updateMetadata(typeId, new BinaryMetadata(typeId, typeName, fieldsMeta, affFieldName0,
                     Collections.singleton(curSchema), false, null), writer.failIfUnregistered());

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
index 5040816..856f41c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
@@ -263,6 +263,12 @@ public class TcpIgniteClient implements IgniteClient {
         }
 
         /** {@inheritDoc} */
+        @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
+            throws BinaryObjectException {
+            throw new UnsupportedOperationException("Can't register metadata locally for thin client.");
+        }
+
+        /** {@inheritDoc} */
         @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
             BinaryType meta = cache.metadata(typeId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 ec88a93..4a6bed4 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
@@ -3823,8 +3823,13 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
             return msg0.needExchange();
         }
 
-        if (msg instanceof DynamicCacheChangeBatch)
-            return cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer);
+        if (msg instanceof DynamicCacheChangeBatch) {
+            boolean changeRequested = cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer);
+
+            ctx.query().onCacheChangeRequested((DynamicCacheChangeBatch)msg);
+
+            return changeRequested;
+        }
 
         if (msg instanceof DynamicCacheChangeFailureMessage)
             cachesInfo.onCacheChangeRequested((DynamicCacheChangeFailureMessage)msg, topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 137db9f..f3078cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -207,6 +207,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                     CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(binaryCtx), failIfUnregistered);
                 }
 
+                @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
+                    throws BinaryObjectException {
+                    CacheObjectBinaryProcessorImpl.this.addMetaLocally(typeId, meta);
+                }
+
                 @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
                     return CacheObjectBinaryProcessorImpl.this.metadata(typeId);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 43310a7..ea588ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import javax.cache.Cache;
+import javax.cache.CacheException;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -34,8 +36,6 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
-import javax.cache.Cache;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -62,16 +62,20 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
+import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
@@ -257,6 +261,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     ctxs.queries().evictDetailMetrics();
             }
         }, QRY_DETAIL_METRICS_EVICTION_FREQ, QRY_DETAIL_METRICS_EVICTION_FREQ);
+
+        registerMetadataForRegisteredCaches();
     }
 
     /** {@inheritDoc} */
@@ -904,6 +910,73 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Register metadata locally for already registered caches.
+     */
+    private void registerMetadataForRegisteredCaches() {
+        for (DynamicCacheDescriptor cacheDescriptor : ctx.cache().cacheDescriptors().values()) {
+            registerBinaryMetadata(cacheDescriptor.cacheConfiguration(), cacheDescriptor.schema());
+        }
+    }
+
+    /**
+     * Handle of cache change request.
+     *
+     * @param batch Dynamic cache change batch request.
+     */
+    public void onCacheChangeRequested(DynamicCacheChangeBatch batch) {
+        for (DynamicCacheChangeRequest req : batch.requests()) {
+            if (!req.start())
+                continue;
+
+            registerBinaryMetadata(req.startCacheConfiguration(), req.schema());
+        }
+    }
+
+    /**
+     * Register binary metadata locally.
+     *
+     * @param ccfg Cache configuration.
+     * @param schema Schema for which register metadata is required.
+     */
+    private void registerBinaryMetadata(CacheConfiguration ccfg, QuerySchema schema) {
+        if (schema != null) {
+            Collection<QueryEntity> qryEntities = schema.entities();
+
+            if (!F.isEmpty(qryEntities)) {
+                boolean binaryEnabled = ctx.cacheObjects().isBinaryEnabled(ccfg);
+
+                if (binaryEnabled) {
+                    for (QueryEntity qryEntity : qryEntities) {
+                        Class<?> keyCls = U.box(U.classForName(qryEntity.findKeyType(), null, true));
+                        Class<?> valCls = U.box(U.classForName(qryEntity.findValueType(), null, true));
+
+                        if (keyCls != null)
+                            registerDescriptorLocallyIfNeeded(keyCls);
+
+                        if (valCls != null)
+                            registerDescriptorLocallyIfNeeded(valCls);
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Register class metadata locally if it didn't do it earlier.
+     *
+     * @param cls Class for which the metadata should be registered.
+     */
+    private void registerDescriptorLocallyIfNeeded(Class<?> cls) {
+        IgniteCacheObjectProcessor cacheObjProc = ctx.cacheObjects();
+
+        if (cacheObjProc instanceof CacheObjectBinaryProcessorImpl) {
+            ((CacheObjectBinaryProcessorImpl)cacheObjProc)
+                .binaryContext()
+                .descriptorForClass(cls, false, false, true);
+        }
+    }
+
+    /**
      * Handle custom discovery message.
      *
      * @param msg Message.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
index c515f81..47138dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/TestCachingMetadataHandler.java
@@ -39,6 +39,12 @@ public class TestCachingMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
+    @Override public void addMetaLocally(int typeId, BinaryType meta, boolean failIfUnregistered)
+        throws BinaryObjectException {
+        addMeta(typeId, meta, failIfUnregistered);
+    }
+
+    /** {@inheritDoc} */
     @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
         return metas.get(typeId);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java
new file mode 100644
index 0000000..d4066c2
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheRegisterMetadataLocallyTest.java
@@ -0,0 +1,287 @@
+/*
+ * 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.cache;
+
+import java.util.Collections;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.managers.discovery.CustomMessageWrapper;
+import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage;
+import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage;
+import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateProposedMessage;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests, that binary metadata is registered correctly during the start without extra request to grid.
+ */
+public class CacheRegisterMetadataLocallyTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String STATIC_CACHE_NAME = "staticCache";
+
+    /** */
+    private static final String DYNAMIC_CACHE_NAME = "dynamicCache";
+
+    /** Holder of sent custom messages. */
+    private final ConcurrentLinkedQueue<Object> customMessages = new ConcurrentLinkedQueue<>();
+
+    /** Holder of sent communication messages. */
+    private final ConcurrentLinkedQueue<Object> communicationMessages = new ConcurrentLinkedQueue<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi() {
+            @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+                if (msg instanceof CustomMessageWrapper)
+                    customMessages.add(((CustomMessageWrapper)msg).delegate());
+                else
+                    customMessages.add(msg);
+
+                super.sendCustomEvent(msg);
+            }
+        });
+
+        cfg.setCommunicationSpi(new TcpCommunicationSpi() {
+            @Override public void sendMessage(ClusterNode node, Message msg,
+                IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
+                if (msg instanceof GridIoMessage)
+                    communicationMessages.add(((GridIoMessage)msg).message());
+
+                super.sendMessage(node, msg, ackC);
+            }
+
+            @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+                if (msg instanceof GridIoMessage)
+                    communicationMessages.add(((GridIoMessage)msg).message());
+
+                super.sendMessage(node, msg);
+            }
+        });
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        if (igniteInstanceName.equals("client"))
+            cfg.setClientMode(true);
+
+        cfg.setCacheConfiguration(cacheConfiguration(STATIC_CACHE_NAME, StaticKey.class, StaticValue.class));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        customMessages.clear();
+        communicationMessages.clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityKeyRegisteredStaticCache() throws Exception {
+        Ignite ignite = startGrid();
+
+        assertEquals("affKey", getAffinityKey(ignite, StaticKey.class));
+        assertEquals("affKey", getAffinityKey(ignite, StaticValue.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityKeyRegisteredDynamicCache() throws Exception {
+        Ignite ignite = startGrid();
+
+        ignite.createCache(cacheConfiguration(DYNAMIC_CACHE_NAME, DynamicKey.class, DynamicValue.class));
+
+        assertEquals("affKey", getAffinityKey(ignite, DynamicKey.class));
+        assertEquals("affKey", getAffinityKey(ignite, DynamicValue.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientFindsValueByAffinityKeyStaticCacheWithoutExtraRequest() throws Exception {
+        Ignite srv = startGrid();
+        IgniteCache<StaticKey, StaticValue> cache = srv.cache(STATIC_CACHE_NAME);
+
+        testClientFindsValueByAffinityKey(cache, new StaticKey(1), new StaticValue(2));
+
+        assertCustomMessages(2); //MetadataUpdateProposedMessage for update schema.
+        assertCommunicationMessages();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientFindsValueByAffinityKeyDynamicCacheWithoutExtraRequest() throws Exception {
+        Ignite srv = startGrid();
+        IgniteCache<DynamicKey, DynamicValue> cache =
+            srv.createCache(cacheConfiguration(DYNAMIC_CACHE_NAME, DynamicKey.class, DynamicValue.class));
+
+        testClientFindsValueByAffinityKey(cache, new DynamicKey(3), new DynamicValue(4));
+
+        //Expected only DynamicCacheChangeBatch for start cache and MetadataUpdateProposedMessage for update schema.
+        assertCustomMessages(3);
+        assertCommunicationMessages();
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     * @param keyCls Key class.
+     * @return Name of affinity key field of the given class.
+     */
+    private <K> String getAffinityKey(Ignite ignite, Class<K> keyCls) {
+        BinaryType binType = ignite.binary().type(keyCls);
+
+        return binType.affinityKeyFieldName();
+    }
+
+    /**
+     * @param cache Cache instance.
+     * @param key Test key.
+     * @param val Test value.
+     * @throws Exception If failed.
+     */
+    private <K, V> void testClientFindsValueByAffinityKey(IgniteCache<K, V> cache, K key, V val) throws Exception {
+        cache.put(key, val);
+
+        assertTrue(cache.containsKey(key));
+
+        Ignite client = startGrid("client");
+
+        IgniteCache<K, V> clientCache = client.cache(cache.getName());
+
+        assertTrue(clientCache.containsKey(key));
+    }
+
+    /**
+     * @param name Cache name.
+     * @param keyCls Key {@link Class}.
+     * @param valCls Value {@link Class}.
+     * @param <K> Key type.
+     * @param <V> Value type.
+     * @return Cache configuration
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, Class<K> keyCls, Class<V> valCls) {
+        CacheConfiguration<K, V> cfg = new CacheConfiguration<>(name);
+        cfg.setQueryEntities(Collections.singleton(new QueryEntity(keyCls, valCls)));
+        return cfg;
+    }
+
+    /**
+     * Expecting that "proposed binary metadata"( {@link org.apache.ignite.internal.processors.marshaller.MappingProposedMessage},
+     * {@link org.apache.ignite.internal.processors.cache.binary.MetadataUpdateProposedMessage}) will be skipped because
+     * it should be register locally during the start.
+     *
+     * @param expMsgCnt Count of expected messages.
+     */
+    private void assertCustomMessages(int expMsgCnt) {
+        assertEquals(customMessages.toString(), expMsgCnt, customMessages.size());
+
+        customMessages.forEach(cm -> assertTrue(cm.toString(), cm instanceof DynamicCacheChangeBatch || cm instanceof MetadataUpdateProposedMessage));
+    }
+
+    /**
+     * Expecting that extra request to binary metadata( {@link MetadataRequestMessage}, {@link MetadataResponseMessage})
+     * will be skipped because it should be register locally during the start.
+     */
+    private void assertCommunicationMessages() {
+        communicationMessages.forEach(cm ->
+            assertFalse(cm.toString(), cm instanceof MetadataRequestMessage || cm instanceof MetadataResponseMessage)
+        );
+    }
+
+    /** */
+    private static class StaticKey {
+        /** */
+        @AffinityKeyMapped
+        private int affKey;
+
+        /**
+         * @param affKey Affinity key.
+         */
+        StaticKey(int affKey) {
+            this.affKey = affKey;
+        }
+    }
+
+    /** */
+    private static class StaticValue {
+        /** */
+        @AffinityKeyMapped
+        private int affKey;
+
+        /**
+         * @param affKey Affinity key.
+         */
+        StaticValue(int affKey) {
+        }
+    }
+
+    /** */
+    private static class DynamicKey {
+        /** */
+        @AffinityKeyMapped
+        private int affKey;
+
+        /**
+         * @param affKey Affinity key.
+         */
+        DynamicKey(int affKey) {
+            this.affKey = affKey;
+        }
+    }
+
+    /** */
+    private static class DynamicValue {
+        /** */
+        @AffinityKeyMapped
+        private int affKey;
+
+        /**
+         * @param affKey Affinity key.
+         */
+        DynamicValue(int affKey) {
+            this.affKey = affKey;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
index 7f1e2e7..5f0e18e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
@@ -629,21 +629,21 @@ public abstract class AbstractSchemaSelfTest extends GridCommonAbstractTest {
     public static class ValueClass {
         /** Field 1. */
         @QuerySqlField
-        private String field1;
+        private Long field1;
 
         /**
          * Constructor.
          *
          * @param field1 Field 1.
          */
-        public ValueClass(String field1) {
+        public ValueClass(Long field1) {
             this.field1 = field1;
         }
 
         /**
          * @return Field 1
          */
-        public String field1() {
+        public Long field1() {
             return field1;
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
index 2d7a636..bfc3881 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java
@@ -60,9 +60,9 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
 
         IgniteCache<KeyClass, ValueClass> cache = client().cache(CACHE_NAME);
 
-        cache.put(new KeyClass(1), new ValueClass("val1"));
-        cache.put(new KeyClass(2), new ValueClass("val2"));
-        cache.put(new KeyClass(3), new ValueClass("val3"));
+        cache.put(new KeyClass(1), new ValueClass(1L));
+        cache.put(new KeyClass(2), new ValueClass(2L));
+        cache.put(new KeyClass(3), new ValueClass(3L));
     }
 
     /** {@inheritDoc} */
@@ -89,14 +89,14 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
                 continue;
 
             List<List<?>> locRes = ignite(i).cache("cache").query(new SqlFieldsQuery("explain select \"id\" from " +
-                "\"cache\".\"ValueClass\" where \"field1\" = 'A'").setLocal(true)).getAll();
+                "\"cache\".\"ValueClass\" where \"field1\" = 1").setLocal(true)).getAll();
 
             assertEquals(F.asList(
                 Collections.singletonList("SELECT\n" +
                     "    \"id\"\n" +
                     "FROM \"cache\".\"ValueClass\"\n" +
-                    "    /* \"cache\".\"idx_1\": \"field1\" = 'A' */\n" +
-                    "WHERE \"field1\" = 'A'")
+                    "    /* \"cache\".\"idx_1\": \"field1\" = 1 */\n" +
+                    "WHERE \"field1\" = 1")
             ), locRes);
         }
 
@@ -106,7 +106,7 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
 
         assertSize(2);
 
-        cache.put(new KeyClass(4), new ValueClass("someVal"));
+        cache.put(new KeyClass(4), new ValueClass(1L));
 
         assertSize(3);
     }
@@ -162,14 +162,14 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
                 continue;
 
             List<List<?>> locRes = ignite(i).cache("cache").query(new SqlFieldsQuery("explain select \"id\" from " +
-                "\"cache\".\"ValueClass\" where \"field1\" = 'A'").setLocal(true)).getAll();
+                "\"cache\".\"ValueClass\" where \"field1\" = 1").setLocal(true)).getAll();
 
             assertEquals(F.asList(
                 Collections.singletonList("SELECT\n" +
                     "    \"id\"\n" +
                     "FROM \"cache\".\"ValueClass\"\n" +
                     "    /* \"cache\".\"ValueClass\".__SCAN_ */\n" +
-                    "WHERE \"field1\" = 'A'")
+                    "WHERE \"field1\" = 1")
             ), locRes);
         }
 
@@ -204,38 +204,39 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
     public void testIndexState() {
         IgniteCache<KeyClass, ValueClass> cache = cache();
 
-        assertColumnValues("val1", "val2", "val3");
+        assertColumnValues(1L, 2L, 3L);
 
         cache.query(new SqlFieldsQuery("CREATE INDEX \"" + IDX_NAME_1_ESCAPED + "\" ON \"" + TBL_NAME_ESCAPED + "\"(\""
             + FIELD_NAME_1_ESCAPED + "\" ASC)"));
 
-        assertColumnValues("val1", "val2", "val3");
+        assertColumnValues(1L, 2L, 3L);
 
         cache.remove(new KeyClass(2));
 
-        assertColumnValues("val1", "val3");
+        assertColumnValues(1L, 3L);
 
-        cache.put(new KeyClass(0), new ValueClass("someVal"));
+        cache.put(new KeyClass(0), new ValueClass(0L));
 
-        assertColumnValues("someVal", "val1", "val3");
+        assertColumnValues(0L, 1L, 3L);
 
         cache.query(new SqlFieldsQuery("DROP INDEX \"" + IDX_NAME_1_ESCAPED + "\""));
 
-        assertColumnValues("someVal", "val1", "val3");
+        assertColumnValues(0L, 1L, 3L);
     }
 
     /**
      * Check that values of {@code field1} match what we expect.
      * @param vals Expected values.
      */
-    private void assertColumnValues(String... vals) {
+    private void assertColumnValues(Long... vals) {
         List<List<?>> expRes = new ArrayList<>(vals.length);
 
-        for (String v : vals)
+        for (Long v : vals)
             expRes.add(Collections.singletonList(v));
 
-        assertEquals(expRes, cache().query(new SqlFieldsQuery("SELECT \"" + FIELD_NAME_1_ESCAPED + "\" FROM \"" +
-            TBL_NAME_ESCAPED + "\" ORDER BY \"id\"")).getAll());
+        List<List<?>> all = cache().query(new SqlFieldsQuery("SELECT \"" + FIELD_NAME_1_ESCAPED + "\" FROM \"" +
+            TBL_NAME_ESCAPED + "\" ORDER BY \"id\"")).getAll();
+        assertEquals(expRes, all);
     }
 
     /**
@@ -245,8 +246,9 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
     private void assertSize(long expSize) {
         assertEquals(expSize, cache().size());
 
-        assertEquals(expSize, cache().query(new SqlFieldsQuery("SELECT COUNT(*) from \"ValueClass\""))
-            .getAll().get(0).get(0));
+        Object actual = cache().query(new SqlFieldsQuery("SELECT COUNT(*) from \"ValueClass\""))
+            .getAll().get(0).get(0);
+        assertEquals(expSize, actual);
     }
 
     /**
@@ -313,8 +315,8 @@ public abstract class H2DynamicIndexAbstractSelfTest extends AbstractSchemaSelfT
         entity.setValueType(ValueClass.class.getName());
 
         entity.addQueryField("id", Long.class.getName(), null);
-        entity.addQueryField(FIELD_NAME_1_ESCAPED, String.class.getName(), null);
-        entity.addQueryField(FIELD_NAME_2_ESCAPED, String.class.getName(), null);
+        entity.addQueryField(FIELD_NAME_1_ESCAPED, Long.class.getName(), null);
+        entity.addQueryField(FIELD_NAME_2_ESCAPED, Long.class.getName(), null);
 
         entity.setKeyFields(Collections.singleton("id"));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bb03444/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index 8517ebb..fae196f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.BinaryTypeMismatchLoggingTest;
+import org.apache.ignite.internal.processors.cache.CacheRegisterMetadataLocallyTest;
 import org.apache.ignite.internal.processors.cache.CacheBinaryKeyConcurrentQueryTest;
 import org.apache.ignite.internal.processors.cache.CacheConfigurationP2PTest;
 import org.apache.ignite.internal.processors.cache.CacheIndexStreamerTest;
@@ -80,6 +81,7 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite {
         suite.addTestSuite(CacheOperationsWithExpirationTest.class);
         suite.addTestSuite(CacheBinaryKeyConcurrentQueryTest.class);
         suite.addTestSuite(CacheQueryFilterExpiredTest.class);
+        suite.addTestSuite(CacheRegisterMetadataLocallyTest.class);
 
         suite.addTestSuite(ClientReconnectAfterClusterRestartTest.class);
 


[03/50] [abbrv] ignite git commit: IGNITE-9877 .NET: Fix AmbiguousMatchException in LINQ under .NET Core 2.1

Posted by sb...@apache.org.
IGNITE-9877 .NET: Fix AmbiguousMatchException in LINQ under .NET Core 2.1


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e0e02aba
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e0e02aba
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e0e02aba

Branch: refs/heads/ignite-gg-14206
Commit: e0e02abaa58ed09ddd7e5eaf528b684dfcc21065
Parents: cb0d1de
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Oct 17 17:20:37 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Oct 17 17:20:37 2018 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |  4 +-
 .../Serializable/AdvancedSerializationTest.cs   |  8 ++--
 .../Binary/Serializable/SqlDmlTest.cs           | 20 ++++-----
 .../Cache/CacheAbstractTest.cs                  | 45 ++++++++++----------
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   | 10 ++---
 .../Client/Cache/CacheTestSsl.cs                |  2 +-
 .../Client/Cache/ScanQueryTest.cs               |  4 +-
 .../Client/IgniteClientConfigurationTest.cs     |  2 +-
 .../Compute/ComputeApiTest.JavaTask.cs          |  2 +-
 .../Services/ServicesTest.cs                    |  2 +-
 .../Impl/Unmanaged/Jni/AppDomains.cs            |  8 ++--
 .../Impl/Unmanaged/Jni/Jvm.cs                   |  2 +-
 .../Impl/Unmanaged/Jni/JvmDll.cs                |  8 ++--
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 10 ++---
 14 files changed, 63 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
index e27f8b7..6550a7f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
@@ -107,7 +107,7 @@
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\LinqTest.cs" Link="ThinClient\Cache\LinqTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\ClientConnectionTest.cs" Link="ThinClient\ClientConnectionTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\ClientTestBase.cs" Link="ThinClient\ClientTestBase.cs" />
-    <Compile Include="..\Apache.Ignite.Core.Tests\Client\EndpointTest.cs" Link="ThinClient\EndpointTest.cs" />    
+    <Compile Include="..\Apache.Ignite.Core.Tests\Client\EndpointTest.cs" Link="ThinClient\EndpointTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\IgniteClientConfigurationTest.cs" Link="ThinClient\IgniteClientConfigurationTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\RawSecureSocketTest.cs" Link="ThinClient\RawSecureSocketTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Compute\ComputeApiTest.cs" Link="Compute\ComputeApiTest.cs" />
@@ -184,7 +184,7 @@
     <PackageReference Include="NUnit" Version="3.8.1" />
     <PackageReference Include="NUnit3TestAdapter" Version="3.9.0" />
     <PackageReference Include="System.Configuration.ConfigurationManager" Version="4.4.0" />
-	
+
 	<ProjectReference Include="..\Apache.Ignite.Core\Apache.Ignite.Core.DotNetCore.csproj" />
 	<ProjectReference Include="..\Apache.Ignite.Linq\Apache.Ignite.Linq.DotNetCore.csproj" />
   </ItemGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/AdvancedSerializationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/AdvancedSerializationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/AdvancedSerializationTest.cs
index 4a10922..d697e17 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/AdvancedSerializationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/AdvancedSerializationTest.cs
@@ -96,7 +96,7 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
             Assert.AreEqual(expectedRes, jobResult.InnerXml);
         }
 
-#if !NETCOREAPP2_0  // AppDomains are not supported in .NET Core
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1  // AppDomains are not supported in .NET Core
         /// <summary>
         /// Tests custom serialization binder.
         /// </summary>
@@ -111,7 +111,7 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
             for (var i = 0; i < count; i++)
             {
                 dynamic val = Activator.CreateInstance(GenerateDynamicType());
-                
+
                 val.Id = i;
                 val.Name = "Name_" + i;
 
@@ -143,7 +143,7 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
                 TypeAttributes.Class | TypeAttributes.Public | TypeAttributes.Serializable);
 
             typeBuilder.DefineField("Id", typeof (int), FieldAttributes.Public);
-            
+
             typeBuilder.DefineField("Name", typeof (string), FieldAttributes.Public);
 
             return typeBuilder.CreateType();
@@ -258,4 +258,4 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
             // No-op.
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/SqlDmlTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/SqlDmlTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/SqlDmlTest.cs
index 0ffd068..005488a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/SqlDmlTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/Serializable/SqlDmlTest.cs
@@ -38,7 +38,7 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
     {
         /** */
         private IIgnite _ignite;
-        
+
         /** */
         private StringBuilder _outSb;
 
@@ -109,7 +109,7 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
             var guid = Guid.NewGuid();
             var insertRes = cache.Query(new SqlFieldsQuery(
                 "insert into SimpleSerializable(_key, Byte, Bool, Short, Int, Long, Float, Double, " +
-                "Decimal, Guid, String) values (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", 
+                "Decimal, Guid, String) values (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)",
                 3, 45, true, 43, 33, 99, 4.5f, 6.7, 9.04m, guid, "bar33")).GetAll();
 
             Assert.AreEqual(1, insertRes.Count);
@@ -163,7 +163,7 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
             Assert.AreEqual("Value was either too large or too small for a UInt32.", ex.Message);
         }
 
-#if !NETCOREAPP2_0  // Console redirect issues on .NET Core
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1  // Console redirect issues on .NET Core
         /// <summary>
         /// Tests the log warning.
         /// </summary>
@@ -196,25 +196,25 @@ namespace Apache.Ignite.Core.Tests.Binary.Serializable
 
             [QuerySqlField]
             public short Short { get; set; }
-            
+
             [QuerySqlField]
             public int Int { get; set; }
-            
+
             [QuerySqlField]
             public long Long { get; set; }
-            
+
             [QuerySqlField]
             public float Float { get; set; }
-            
+
             [QuerySqlField]
             public double Double { get; set; }
-            
+
             [QuerySqlField]
             public decimal Decimal { get; set; }
-            
+
             [QuerySqlField]
             public Guid Guid { get; set; }
-            
+
             [QuerySqlField]
             public string String { get; set; }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 02ed39d..3c74ce4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -39,7 +39,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     /// Base cache test.
     /// </summary>
     [SuppressMessage("ReSharper", "UnusedVariable")]
-    public abstract class CacheAbstractTest 
+    public abstract class CacheAbstractTest
     {
         /// <summary>
         /// Fixture setup.
@@ -91,7 +91,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         /// </summary>
         [TearDown]
         public void AfterTest() {
-            for (int i = 0; i < GridCount(); i++) 
+            for (int i = 0; i < GridCount(); i++)
                 Cache(i).WithKeepBinary<object, object>().RemoveAll();
 
             for (int i = 0; i < GridCount(); i++)
@@ -213,7 +213,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.IsTrue(cache.ContainsKey(key));
             Assert.IsFalse(cache.ContainsKey(-1));
         }
-        
+
         [Test]
         public void TestContainsKeys()
         {
@@ -229,7 +229,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             Assert.IsFalse(cache.ContainsKeys(keys.Concat(new[] {int.MaxValue})));
         }
-        
+
         [Test]
         public void TestPeek()
         {
@@ -260,11 +260,11 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             Assert.AreEqual(1, cache.Get(1));
             Assert.AreEqual(2, cache.Get(2));
-            
+
             Assert.Throws<KeyNotFoundException>(() => cache.Get(3));
 
             int value;
-            
+
             Assert.IsTrue(cache.TryGet(1, out value));
             Assert.AreEqual(1, value);
 
@@ -282,7 +282,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             cache.Put(1, 1);
             cache.Put(2, 2);
-            
+
             Assert.AreEqual(1, cache.Get(1));
             Assert.AreEqual(2, cache.Get(2));
             Assert.IsFalse(cache.ContainsKey(3));
@@ -376,11 +376,11 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(1, cache.Get(1));
 
             Assert.IsFalse(cache.GetAndRemove(0).Success);
-            
+
             Assert.AreEqual(1, cache.GetAndRemove(1).Value);
 
             Assert.IsFalse(cache.GetAndRemove(1).Success);
-            
+
             Assert.IsFalse(cache.ContainsKey(1));
         }
 
@@ -653,10 +653,9 @@ namespace Apache.Ignite.Core.Tests.Cache
         /// Expiry policy tests.
         /// </summary>
         [Test]
+        [Ignore("https://issues.apache.org/jira/browse/IGNITE-8983")]
         public void TestWithExpiryPolicy()
         {
-            Assert.Fail("https://issues.apache.org/jira/browse/IGNITE-8983");
-
             TestWithExpiryPolicy((cache, policy) => cache.WithExpiryPolicy(policy), true);
         }
 
@@ -680,11 +679,11 @@ namespace Apache.Ignite.Core.Tests.Cache
         /// <summary>
         /// Expiry policy tests.
         /// </summary>
-        private void TestWithExpiryPolicy(Func<ICache<int, int>, IExpiryPolicy, ICache<int, int>> withPolicyFunc, 
+        private void TestWithExpiryPolicy(Func<ICache<int, int>, IExpiryPolicy, ICache<int, int>> withPolicyFunc,
             bool origCache)
         {
             ICache<int, int> cache0 = Cache(0);
-            
+
             int key0;
             int key1;
 
@@ -698,7 +697,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                 key0 = GetPrimaryKeyForCache(cache0);
                 key1 = GetPrimaryKeyForCache(Cache(1));
             }
-            
+
             // Test unchanged expiration.
             ICache<int, int> cache = withPolicyFunc(cache0, new ExpiryPolicy(null, null, null));
             cache0 = origCache ? cache0 : cache;
@@ -787,7 +786,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.IsFalse(cache0.ContainsKey(key0));
             Assert.IsFalse(cache0.ContainsKey(key1));
         }
-        
+
         /// <summary>
         /// Expiry policy tests for zero and negative expiry values.
         /// </summary>
@@ -796,7 +795,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         public void TestWithExpiryPolicyZeroNegative()
         {
             ICache<int, int> cache0 = Cache(0);
-            
+
             int key0;
             int key1;
 
@@ -838,7 +837,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             cache0.RemoveAll(new List<int> { key0, key1 });
 
             // Test negative expiration.
-            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromMilliseconds(-100), 
+            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromMilliseconds(-100),
                 TimeSpan.FromMilliseconds(-100), TimeSpan.FromMilliseconds(-100)));
 
             cache.Put(key0, key0);
@@ -1714,7 +1713,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             Assert.IsNull(err);
         }
-        
+
         /**
          * Test tries to provoke garbage collection for .Net future before it was completed to verify
          * futures pinning works.
@@ -2041,7 +2040,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             foreach (var nearKey in nearKeys.Take(3))
                 Assert.AreNotEqual(0, cache.Get(nearKey));
         }
-        
+
         [Test]
         public void TestSerializable()
         {
@@ -2180,7 +2179,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             var results = res.OrderBy(x => x.Key).Select(x => x.Result);
             var expectedResults = entries.OrderBy(x => x.Key).Select(x => x.Value + arg);
-            
+
             Assert.IsTrue(results.SequenceEqual(expectedResults));
 
             var resultEntries = cache.GetAll(entries.Keys);
@@ -2197,20 +2196,20 @@ namespace Apache.Ignite.Core.Tests.Cache
             res = cache.InvokeAll(entries.Keys, new T {Exists = false}, arg);
 
             Assert.IsTrue(res.All(x => x.Result == arg));
-            Assert.IsTrue(cache.GetAll(entries.Keys).All(x => x.Value == arg)); 
+            Assert.IsTrue(cache.GetAll(entries.Keys).All(x => x.Value == arg));
 
             // Test exceptions
             var errKey = entries.Keys.Reverse().Take(5).Last();
 
             TestInvokeAllException(cache, entries, new T { ThrowErr = true, ThrowOnKey = errKey }, arg, errKey);
-            TestInvokeAllException(cache, entries, new T { ThrowErrBinarizable = true, ThrowOnKey = errKey }, 
+            TestInvokeAllException(cache, entries, new T { ThrowErrBinarizable = true, ThrowOnKey = errKey },
                 arg, errKey);
             TestInvokeAllException(cache, entries, new T { ThrowErrNonSerializable = true, ThrowOnKey = errKey },
                 arg, errKey, "ExpectedException");
 
         }
 
-        private static void TestInvokeAllException<T>(ICache<int, int> cache, Dictionary<int, int> entries, 
+        private static void TestInvokeAllException<T>(ICache<int, int> cache, Dictionary<int, int> entries,
             T processor, int arg, int errKey, string exceptionText = null) where T : AddArgCacheEntryProcessor
         {
             var res = cache.InvokeAll(entries.Keys, processor, arg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
index cb89a5b..628f35c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
@@ -66,7 +66,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
 
             CheckFunc(x => x.Trim(), strings);
 
-#if !NETCOREAPP2_0  // Trim is not supported on .NET Core
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1  // Trim is not supported on .NET Core
             CheckFunc(x => x.Trim('P'), strings);
             var toTrim = new[] { 'P' };
             CheckFunc(x => x.Trim(toTrim), strings);
@@ -89,9 +89,9 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
             CheckFunc(x => Regex.Replace(x, @"son.\d", "kele!", RegexOptions.None), strings);
             CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.IgnoreCase), strings);
             CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.Multiline), strings);
-            CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.IgnoreCase | RegexOptions.Multiline), 
+            CheckFunc(x => Regex.Replace(x, @"person.\d", "akele!", RegexOptions.IgnoreCase | RegexOptions.Multiline),
                 strings);
-            var notSupportedException = Assert.Throws<NotSupportedException>(() => CheckFunc(x => 
+            var notSupportedException = Assert.Throws<NotSupportedException>(() => CheckFunc(x =>
                 Regex.IsMatch(x, @"^person\d", RegexOptions.IgnoreCase | RegexOptions.CultureInvariant), strings));
             Assert.AreEqual("RegexOptions.CultureInvariant is not supported", notSupportedException.Message);
 
@@ -100,7 +100,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
             CheckFunc(x => Regex.IsMatch(x, @"^person_9\d", RegexOptions.IgnoreCase), strings);
             CheckFunc(x => Regex.IsMatch(x, @"^Person_9\d", RegexOptions.Multiline), strings);
             CheckFunc(x => Regex.IsMatch(x, @"^person_9\d", RegexOptions.IgnoreCase | RegexOptions.Multiline), strings);
-            notSupportedException = Assert.Throws<NotSupportedException>(() => CheckFunc(x => 
+            notSupportedException = Assert.Throws<NotSupportedException>(() => CheckFunc(x =>
                 Regex.IsMatch(x, @"^person_9\d",RegexOptions.IgnoreCase | RegexOptions.CultureInvariant), strings));
             Assert.AreEqual("RegexOptions.CultureInvariant is not supported", notSupportedException.Message);
 
@@ -114,4 +114,4 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
             CheckFunc(x => x + 10, strings);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTestSsl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTestSsl.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTestSsl.cs
index 0f55ce5..f674232 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTestSsl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTestSsl.cs
@@ -54,7 +54,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
                     CertificatePassword = "123456",
                     SkipServerCertificateValidation = true,
                     CheckCertificateRevocation = true,
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
                     SslProtocols = SslProtocols.Tls
 #else
                     SslProtocols = SslProtocols.Tls12

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
index 71c8f0f..18ced28 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
@@ -124,7 +124,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
                 var single = clientCache.Query(new ScanQuery<int, Person>(new PersonKeyFilter(3))).Single();
                 Assert.AreEqual(3, single.Key);
 
-#if !NETCOREAPP2_0   // Serializing delegates is not supported on this platform.
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1   // Serializing delegates is not supported on this platform.
                 // Multiple results.
                 var res = clientCache.Query(new ScanQuery<int, Person>(new PersonFilter(x => x.Name.Length == 1)))
                     .ToList();
@@ -157,7 +157,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         }
 
 
-#if !NETCOREAPP2_0   // Serializing delegates and exceptions is not supported on this platform.
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1   // Serializing delegates and exceptions is not supported on this platform.
         /// <summary>
         /// Tests the exception in filter.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/IgniteClientConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/IgniteClientConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/IgniteClientConfigurationTest.cs
index 3d55f4c..a113193 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/IgniteClientConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/IgniteClientConfigurationTest.cs
@@ -196,7 +196,7 @@ namespace Apache.Ignite.Core.Tests.Client
             }
         }
 
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
         /// <summary>
         /// Tests the schema validation.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.JavaTask.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.JavaTask.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.JavaTask.cs
index 1f5c3a3..be0a203 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.JavaTask.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.JavaTask.cs
@@ -316,7 +316,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
                 Assert.AreEqual(val, binRes.GetField<long>("Field"));
 
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
                 var dotNetBin = _grid1.GetBinary().ToBinary<BinaryObject>(res);
 
                 Assert.AreEqual(dotNetBin.Header.HashCode, ((BinaryObject)binRes).Header.HashCode);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index 81c3652..6bea9b2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -1018,7 +1018,7 @@ namespace Apache.Ignite.Core.Tests.Services
         /// </summary>
         private IgniteConfiguration GetConfiguration(string springConfigUrl)
         {
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
             if (!CompactFooter)
             {
                 springConfigUrl = Compute.ComputeApiTestFullFooter.ReplaceFooterSetting(springConfigUrl);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/AppDomains.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/AppDomains.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/AppDomains.cs
index e98796b..64c3665 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/AppDomains.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/AppDomains.cs
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
 namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
 {
     using System;
@@ -65,12 +65,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
             {
                 throw new IgniteException("Failed to get default AppDomain. Cannot create meta host: " + hr);
             }
-            
+
             var host = (ICLRMetaHost) objHost;
             var vers = Environment.Version;
             var versString = string.Format("v{0}.{1}.{2}", vers.Major, vers.Minor, vers.Build);
             var runtime = (ICLRRuntimeInfo) host.GetRuntime(versString, ref IID_CLRRuntimeInfo);
-            
+
             bool started;
             uint flags;
             runtime.IsStarted(out started, out flags);
@@ -133,4 +133,4 @@ namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
         }
     }
 }
-#endif
\ No newline at end of file
+#endif

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/Jvm.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/Jvm.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/Jvm.cs
index 5c781b5..4e04a80 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/Jvm.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/Jvm.cs
@@ -107,7 +107,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
         /// </summary>
         private static Callbacks GetCallbacksFromDefaultDomain()
         {
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
             // JVM exists once per process, and JVM callbacks exist once per process.
             // We should register callbacks ONLY from the default AppDomain (which can't be unloaded).
             // Non-default appDomains should delegate this logic to the default one.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/JvmDll.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/JvmDll.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/JvmDll.cs
index ef161f4..682226d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/JvmDll.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/Jni/JvmDll.cs
@@ -172,8 +172,8 @@ namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
 
         /// <summary>
         /// Gets the default JVM init args.
-        /// Before calling this function, native code must set the vm_args->version field to the JNI version 
-        /// it expects the VM to support. After this function returns, vm_args->version will be set 
+        /// Before calling this function, native code must set the vm_args->version field to the JNI version
+        /// it expects the VM to support. After this function returns, vm_args->version will be set
         /// to the actual JNI version the VM supports.
         /// </summary>
         public unsafe JniResult GetDefaultJvmInitArgs(JvmInitArgs* args)
@@ -305,7 +305,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
         /// </summary>
         private static IEnumerable<KeyValuePair<string, string>> GetJvmDllPathsWindows()
         {
-#if !NETCOREAPP2_0
+#if !NETCOREAPP2_0 && !NETCOREAPP2_1
             if (!Os.IsWindows)
             {
                 yield break;
@@ -446,4 +446,4 @@ namespace Apache.Ignite.Core.Impl.Unmanaged.Jni
             internal static extern JniResult JNI_GetDefaultJavaVMInitArgs(JvmInitArgs* args);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0e02aba/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index 84bd98f..375c7a8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -54,7 +54,7 @@ namespace Apache.Ignite.Linq.Impl
         {
             GetStringMethod("ToLower", new Type[0], GetFunc("lower")),
             GetStringMethod("ToUpper", new Type[0], GetFunc("upper")),
-            GetStringMethod("Contains", del: (e, v) => VisitSqlLike(e, v, "'%' || ? || '%'")),
+            GetStringMethod("Contains", new[] {typeof (string)}, (e, v) => VisitSqlLike(e, v, "'%' || ? || '%'")),
             GetStringMethod("StartsWith", new[] {typeof (string)}, (e, v) => VisitSqlLike(e, v, "? || '%'")),
             GetStringMethod("EndsWith", new[] {typeof (string)}, (e, v) => VisitSqlLike(e, v, "'%' || ?")),
             GetStringMethod("IndexOf", new[] {typeof (string)}, GetFunc("instr", -1)),
@@ -72,7 +72,7 @@ namespace Apache.Ignite.Linq.Impl
             GetStringMethod("PadRight", "rpad", typeof (int), typeof (char)),
 
             GetRegexMethod("Replace", "regexp_replace", typeof (string), typeof (string), typeof (string)),
-            GetRegexMethod("Replace", "regexp_replace", typeof (string), typeof (string), typeof (string), 
+            GetRegexMethod("Replace", "regexp_replace", typeof (string), typeof (string), typeof (string),
                 typeof(RegexOptions)),
             GetRegexMethod("IsMatch", "regexp_like", typeof (string), typeof (string)),
             GetRegexMethod("IsMatch", "regexp_like", typeof (string), typeof (string), typeof(RegexOptions)),
@@ -205,7 +205,7 @@ namespace Apache.Ignite.Linq.Impl
         /// <summary>
         /// Visits the instance function.
         /// </summary>
-        private static void VisitFunc(MethodCallExpression expression, CacheQueryExpressionVisitor visitor, 
+        private static void VisitFunc(MethodCallExpression expression, CacheQueryExpressionVisitor visitor,
             string func, string suffix, params int[] adjust)
         {
             visitor.ResultBuilder.Append(func).Append("(");
@@ -358,7 +358,7 @@ namespace Apache.Ignite.Linq.Impl
         private static KeyValuePair<MethodInfo, VisitMethodDelegate> GetParameterizedTrimMethod(string name,
             string sqlName)
         {
-            return GetMethod(typeof(string), name, new[] {typeof(char[])}, 
+            return GetMethod(typeof(string), name, new[] {typeof(char[])},
                 (e, v) => VisitParameterizedTrimFunc(e, v, sqlName));
         }
 
@@ -379,4 +379,4 @@ namespace Apache.Ignite.Linq.Impl
             return GetMathMethod(name, name, argTypes);
         }
     }
-}
\ No newline at end of file
+}