You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/10/13 17:42:46 UTC

[01/50] [abbrv] ignite git commit: .NET: Remove empty file (merge byproduct)

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.1.5-p1 [created] cc87a6215


.NET: Remove empty file (merge byproduct)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 542f2c2b8b93529bde63a76eb8833d6fd2c61020
Parents: 6c416d5
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Sep 19 13:30:01 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Sep 19 13:30:01 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs | 0
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/542f2c2b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
deleted file mode 100644
index e69de29..0000000


[17/50] [abbrv] ignite git commit: IGNITE-6342 Fixed exchange hanging on eviction when partition is lost

Posted by yz...@apache.org.
IGNITE-6342 Fixed exchange hanging on eviction when partition is lost


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: b6d570fb0d5cc5638f857a5aa463600a942fbba0
Parents: 931320f
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Sep 21 16:37:42 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Sep 21 16:39:28 2017 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         | 18 +++---
 .../dht/GridDhtPartitionTopologyImpl.java       | 13 +++-
 .../GridDhtPartitionsExchangeFuture.java        |  2 +-
 .../preloader/GridDhtPartitionsFullMessage.java |  2 +-
 .../persistence/GridCacheOffheapManager.java    |  2 +-
 .../IgniteCacheDatabaseSharedManager.java       |  2 +-
 .../cache/persistence/tree/io/IOVersions.java   |  3 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  3 +-
 .../IgnitePdsContinuousRestartTest.java         | 45 ++++++++++++-
 ...ousRestartTestWithSharedGroupAndIndexes.java | 67 ++++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |  3 +
 11 files changed, 144 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/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 15473f1..18815ac 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
@@ -1265,7 +1265,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         "(all nodes in topology should have identical value) " +
                         "[locPreferIpV4=" + locPreferIpV4 + ", rmtPreferIpV4=" + rmtPreferIpV4 +
                         ", locId8=" + U.id8(locNode.id()) + ", rmtId8=" + U.id8(n.id()) +
-                        ", rmtAddrs=" + U.addressesAsString(n) + ']',
+                        ", rmtAddrs=" + U.addressesAsString(n) + ", rmtNode=" + U.toShortString(n) + "]",
                         "Local and remote 'java.net.preferIPv4Stack' system properties do not match.");
 
                 ipV4Warned = true;
@@ -1280,7 +1280,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     throw new IgniteCheckedException("Remote node has deployment mode different from local " +
                         "[locId8=" + U.id8(locNode.id()) + ", locMode=" + locMode +
                         ", rmtId8=" + U.id8(n.id()) + ", rmtMode=" + rmtMode +
-                        ", rmtAddrs=" + U.addressesAsString(n) + ']');
+                        ", rmtAddrs=" + U.addressesAsString(n) + ", rmtNode=" + U.toShortString(n) + "]");
 
                 boolean rmtP2pEnabled = n.attribute(ATTR_PEER_CLASSLOADING);
 
@@ -1288,7 +1288,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     throw new IgniteCheckedException("Remote node has peer class loading enabled flag different from" +
                         " local [locId8=" + U.id8(locNode.id()) + ", locPeerClassLoading=" + locP2pEnabled +
                         ", rmtId8=" + U.id8(n.id()) + ", rmtPeerClassLoading=" + rmtP2pEnabled +
-                        ", rmtAddrs=" + U.addressesAsString(n) + ']');
+                        ", rmtAddrs=" + U.addressesAsString(n) + ", rmtNode=" + U.toShortString(n) + "]");
             }
 
             Boolean rmtMarshUseDfltSuid = n.attribute(ATTR_MARSHALLER_USE_DFLT_SUID);
@@ -1302,7 +1302,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     "[locMarshUseDfltSuid=" + locMarshUseDfltSuid + ", rmtMarshUseDfltSuid=" + rmtMarshUseDfltSuid +
                     ", locNodeAddrs=" + U.addressesAsString(locNode) +
                     ", rmtNodeAddrs=" + U.addressesAsString(n) +
-                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']');
+                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ", rmtNode=" + U.toShortString(n) + "]");
             }
 
             Boolean rmtMarshStrSerVer2 = n.attribute(ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2);
@@ -1316,7 +1316,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     "[locMarshStrSerVer2=" + locMarshStrSerVer2 + ", rmtMarshStrSerVer2=" + rmtMarshStrSerVer2 +
                     ", locNodeAddrs=" + U.addressesAsString(locNode) +
                     ", rmtNodeAddrs=" + U.addressesAsString(n) +
-                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']');
+                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ", rmtNode=" + U.toShortString(n) + "]");
             }
 
             boolean rmtLateAssign = n.attribute(ATTR_LATE_AFFINITY_ASSIGNMENT);
@@ -1327,7 +1327,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     ", locDelayAssign=" + locDelayAssign +
                     ", rmtId8=" + U.id8(n.id()) +
                     ", rmtLateAssign=" + rmtLateAssign +
-                    ", rmtAddrs=" + U.addressesAsString(n) + ']');
+                    ", rmtAddrs=" + U.addressesAsString(n) + ", rmtNode=" + U.toShortString(n) + "]");
             }
 
             Boolean rmtSrvcCompatibilityEnabled = n.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
@@ -1341,7 +1341,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     ", rmtSrvcCompatibilityEnabled=" + rmtSrvcCompatibilityEnabled +
                     ", locNodeAddrs=" + U.addressesAsString(locNode) +
                     ", rmtNodeAddrs=" + U.addressesAsString(n) +
-                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']');
+                    ", locNodeId=" + locNode.id() + ", rmtNode=" + U.toShortString(n) + "]");
             }
 
             if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0
@@ -1358,7 +1358,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         ", rmtSecurityCompatibilityEnabled=" + rmtSecurityCompatibilityEnabled +
                         ", locNodeAddrs=" + U.addressesAsString(locNode) +
                         ", rmtNodeAddrs=" + U.addressesAsString(n) +
-                        ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']');
+                        ", locNodeId=" + locNode.id() + ", rmtNode=" + U.toShortString(n) + "]");
                 }
             }
 
@@ -1372,7 +1372,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     ", locNodeAddrs=" + U.addressesAsString(locNode) +
                     ", rmtNodeAddrs=" + U.addressesAsString(n) +
                     ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ", " +
-                    ", rmtNodeVer" + n.version() + ']');
+                    ", rmtNodeVer" + n.version() + ", rmtNode=" + U.toShortString(n) + "]");
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index f7f71a1..cad21d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1786,11 +1786,22 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 for (Integer part : lost) {
                     long updSeq = updateSeq.incrementAndGet();
 
-                    GridDhtLocalPartition locPart = localPartition(part, resTopVer, false);
+                    GridDhtLocalPartition locPart = localPartition(part, resTopVer, false, true);
 
                     if (locPart != null) {
                         boolean marked = plc == PartitionLossPolicy.IGNORE ? locPart.own() : locPart.markLost();
 
+                        if (!marked && locPart.state() == RENTING)
+                            try {
+                                //TODO https://issues.apache.org/jira/browse/IGNITE-6433
+                                locPart.tryEvict();
+                                locPart.rent(false).get();
+                            }
+                            catch (IgniteCheckedException e) {
+                                U.error(log, "Failed to wait for RENTING partition eviction after partition LOST event",
+                                    e);
+                            }
+
                         if (marked)
                             updateLocal(locPart.id(), locPart.state(), updSeq, resTopVer);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/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 240b5f0..0ec0f82 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
@@ -1415,7 +1415,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                 cacheCtx.continuousQueries().flushBackupQueue(res);
             }
-       }
+        }
 
         if (err == null) {
             if (centralizedAff) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index edbfc23..4a449d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -49,7 +49,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** */
+    /** grpId -> FullMap */
     @GridToStringInclude
     @GridDirectTransient
     private Map<Integer, GridDhtPartitionFullMap> parts;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/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 5cd12af..4f48bbf 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
@@ -194,7 +194,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                     else {
                         // localPartition will not acquire writeLock here because create=false.
                         GridDhtLocalPartition part = grp.topology().localPartition(store.partId(),
-                            AffinityTopologyVersion.NONE, false);
+                            AffinityTopologyVersion.NONE, false, true);
 
                         if (part != null && part.state() != GridDhtPartitionState.EVICTED)
                             state = part.state();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/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 9123b42..63ae8bc 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
@@ -735,7 +735,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /** {@inheritDoc} */
     @Override public boolean checkpointLockIsHeldByThread() {
-        return false;
+        return true;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java
index 48b0da1..d74d344 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java
@@ -79,6 +79,9 @@ public final class IOVersions<V extends PageIO> {
      * @return IO.
      */
     public V forVersion(int ver) {
+        if (ver == 0)
+            throw new IllegalStateException("Failed to get page IO instance (page content is corrupted)");
+
         return vers[ver - 1];
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 971281f..3c974ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -928,7 +928,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                         " node is configured to use loopback address, but " + secondNode + " node is not " +
                         "(consider changing 'localAddress' configuration parameter) " +
                         "[locNodeAddrs=" + U.addressesAsString(locNode) + ", rmtNodeAddrs=" +
-                        U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
+                        U.addressesAsString(msg.addresses(), msg.hostNames()) +
+                        ", creatorNodeId=" + msg.creatorNodeId() + ']');
                 }
                 else
                     LT.warn(log, "Node has not been connected to topology and will repeat join process. " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
index 238e1d1..df62f3a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java
@@ -22,6 +22,7 @@ import java.util.Random;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -54,6 +55,25 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest {
     /** */
     public static final String CACHE_NAME = "cache1";
 
+    /** Checkpoint delay. */
+    private volatile int checkpointDelay = -1;
+
+    /** */
+    private boolean cancel = false;
+
+    /**
+     * Default constructor.
+     */
+    public IgnitePdsContinuousRestartTest() {
+    }
+
+    /**
+     * @param cancel Cancel.
+     */
+    public IgnitePdsContinuousRestartTest(boolean cancel) {
+        this.cancel = cancel;
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -84,6 +104,7 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest {
         cfg.setPersistentStoreConfiguration(
             new PersistentStoreConfiguration()
                 .setWalMode(WALMode.LOG_ONLY)
+                .setCheckpointingFrequency(checkpointDelay)
         );
 
         return cfg;
@@ -195,6 +216,27 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testRebalncingDuringLoad_10_10_1_1() throws Exception {
+        checkRebalancingDuringLoad(10, 10, 1, 1);
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testRebalncingDuringLoad_10_500_8_16() throws Exception {
+        checkRebalancingDuringLoad(10, 500, 8, 16);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TimeUnit.MINUTES.toMillis(3);
+    }
+
+    /**
      * @throws Exception if failed.
      */
     private void checkRebalancingDuringLoad(
@@ -203,6 +245,7 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest {
         int threads,
         final int batch
     ) throws Exception {
+        this.checkpointDelay = checkpointDelay;
 
         startGrids(GRID_CNT);
 
@@ -245,7 +288,7 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest {
         while (System.currentTimeMillis() < end) {
             int idx = rnd.nextInt(GRID_CNT - 1) + 1;
 
-            stopGrid(idx);
+            stopGrid(idx, cancel);
 
             U.sleep(restartDelay);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes.java
new file mode 100644
index 0000000..110e677
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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;
+
+/**
+ * Adding shared group and indexes to testing. It would impact how we evict partitions.
+ */
+public class IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes extends IgnitePdsContinuousRestartTest {
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache 2 singleton group name. */
+    public static final String CACHE_GROUP_NAME = "Group2";
+
+    /**
+     * Default constructor.
+     */
+    public IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes() {
+        super(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoverySpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
+        discoverySpi.setIpFinder(ipFinder);
+
+        CacheConfiguration ccfg2 = new CacheConfiguration();
+
+        ccfg2.setName(CACHE_NAME);
+        ccfg2.setGroupName(CACHE_GROUP_NAME);
+        ccfg2.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg2.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg2.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg2.setIndexedTypes(Integer.class, Integer.class);
+        ccfg2.setBackups(2);
+
+        cfg.setCacheConfiguration(ccfg2);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b6d570fb/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 4f4dedf..ab8ff81 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
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTest;
+import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistenceMetricsSelfTest;
@@ -64,6 +65,8 @@ public class IgnitePdsTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgnitePdsContinuousRestartTest.class);
 
+        suite.addTestSuite(IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes.class);
+
         suite.addTestSuite(IgnitePersistentStoreDataStructuresTest.class);
 
         // Failover test


[25/50] [abbrv] ignite git commit: IGNITE-6460 Fixed consistentId used in lightweight ClusterNode. Fixes #2707

Posted by yz...@apache.org.
IGNITE-6460 Fixed consistentId used in lightweight ClusterNode. Fixes #2707


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 49a3631937b09c2fa795d29411635b3b5f82f15a
Parents: e82f077
Author: Eduard Shangareev <ed...@gmail.com>
Authored: Thu Sep 21 18:55:00 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Sep 21 18:57:33 2017 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/49a36319/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index b2797a40..20fb6c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -652,7 +652,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
 
     /**
      * IMPORTANT!
-     * Only purpose of this constructor is creating node which contains only necessary data to store on disc
+     * Only purpose of this constructor is creating node which contains necessary data to store on disc only
      * @param node to copy data from
      */
     public TcpDiscoveryNode(
@@ -667,6 +667,6 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         this.daemon = node.isDaemon();
         this.clientRouterNodeId = node.isClient() ? node.id() : null;
 
-        attrs = Collections.emptyMap();
+        attrs = Collections.singletonMap(ATTR_NODE_CONSISTENT_ID, consistentId);
     }
 }


[48/50] [abbrv] ignite git commit: IGNITE-6493 fix IgnitePdsWalTlbTest.testWalDirectOutOfMemory() incorrect usage dataStreamer, must be in try-with-resource

Posted by yz...@apache.org.
IGNITE-6493 fix IgnitePdsWalTlbTest.testWalDirectOutOfMemory() incorrect usage dataStreamer, must be in try-with-resource

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

(cherry picked from commit 21cc7a4)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: d3a672e8b99202589aba8ffa4a9c1561a68a985d
Parents: 251a333
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon Sep 25 13:28:42 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Sep 25 13:34:01 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d3a672e8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
index fd0fd34..8b3cbf8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
@@ -113,9 +113,7 @@ public class IgnitePdsWalTlbTest extends GridCommonAbstractTest {
 
         boolean locked = true;
 
-        try {
-            IgniteDataStreamer<Integer, Integer> streamer = ig.dataStreamer(CACHE_NAME);
-
+        try (IgniteDataStreamer<Integer, Integer> streamer = ig.dataStreamer(CACHE_NAME)) {
             for (int i = 0; i < 100_000; i++) {
                 streamer.addData(i, 1);
 


[30/50] [abbrv] ignite git commit: IGNITE-6213 Removed locDepOwner flag and marked as deprecated - Fixes #2723.

Posted by yz...@apache.org.
IGNITE-6213 Removed locDepOwner flag and marked as deprecated - Fixes #2723.

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

(cherry picked from commit 1649c53)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 73e1578370c35f3428b097f2f185adf34dace881
Parents: f08127d
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Fri Sep 22 12:11:15 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 12:35:17 2017 +0300

----------------------------------------------------------------------
 .../internal/GridEventConsumeHandler.java       |  9 +++-
 .../managers/deployment/GridDeploymentInfo.java |  1 +
 .../deployment/GridDeploymentInfoBean.java      | 21 +++-----
 .../cache/GridCacheDeploymentManager.java       | 54 ++++++--------------
 .../processors/cache/GridCacheIoManager.java    |  9 +++-
 .../continuous/CacheContinuousQueryHandler.java |  9 +++-
 6 files changed, 45 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/73e15783/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 54e3d7f..ac568f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -321,8 +321,13 @@ class GridEventConsumeHandler implements GridContinuousHandler {
                     GridDeploymentInfo depInfo = wrapper.depInfo;
 
                     if (depInfo != null) {
-                        depMgr.p2pContext(nodeId, depInfo.classLoaderId(), depInfo.userVersion(), depInfo.deployMode(),
-                            depInfo.participants(), depInfo.localDeploymentOwner());
+                        depMgr.p2pContext(
+                            nodeId,
+                            depInfo.classLoaderId(),
+                            depInfo.userVersion(),
+                            depInfo.deployMode(),
+                            depInfo.participants()
+                        );
                     }
 
                     ldr = depMgr.globalLoader();

http://git-wip-us.apache.org/repos/asf/ignite/blob/73e15783/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfo.java
index b14fcce..3396d82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfo.java
@@ -49,6 +49,7 @@ public interface GridDeploymentInfo {
     /**
      * @return Local deployment ownership flag.
      */
+    @Deprecated
     public boolean localDeploymentOwner();
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/73e15783/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
index 68f45da..7f58ce3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
@@ -52,6 +52,7 @@ public class GridDeploymentInfoBean implements Message, GridDeploymentInfo, Exte
     private String userVer;
 
     /** */
+    @Deprecated // Left for backward compatibility only.
     private boolean locDepOwner;
 
     /** Node class loader participant map. */
@@ -71,15 +72,17 @@ public class GridDeploymentInfoBean implements Message, GridDeploymentInfo, Exte
      * @param userVer User version.
      * @param depMode Deployment mode.
      * @param participants Participants.
-     * @param locDepOwner Local deployment owner flag.
      */
-    public GridDeploymentInfoBean(IgniteUuid clsLdrId, String userVer, DeploymentMode depMode,
-        Map<UUID, IgniteUuid> participants, boolean locDepOwner) {
+    public GridDeploymentInfoBean(
+        IgniteUuid clsLdrId,
+        String userVer,
+        DeploymentMode depMode,
+        Map<UUID, IgniteUuid> participants
+    ) {
         this.clsLdrId = clsLdrId;
         this.depMode = depMode;
         this.userVer = userVer;
         this.participants = participants;
-        this.locDepOwner = locDepOwner;
     }
 
     /**
@@ -89,7 +92,6 @@ public class GridDeploymentInfoBean implements Message, GridDeploymentInfo, Exte
         clsLdrId = dep.classLoaderId();
         depMode = dep.deployMode();
         userVer = dep.userVersion();
-        locDepOwner = dep.localDeploymentOwner();
         participants = dep.participants();
     }
 
@@ -123,15 +125,6 @@ public class GridDeploymentInfoBean implements Message, GridDeploymentInfo, Exte
         return participants;
     }
 
-    /**
-     * Sets local deployment ownership flag.
-     *
-     * @param locDepOwner Local deployment ownership flag.
-     */
-    public void localDeploymentOwner(boolean locDepOwner) {
-        this.locDepOwner = locDepOwner;
-    }
-
     /** {@inheritDoc} */
     @Override public void onAckReceived() {
         // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/73e15783/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index a1eb706..c40ed98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -82,9 +82,6 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
     /** Local deployment. */
     private final AtomicReference<GridDeployment> locDep = new AtomicReference<>();
 
-    /** Local deployment ownership flag. */
-    private volatile boolean locDepOwner;
-
     /** */
     private final ThreadLocal<Boolean> ignoreOwnership = new ThreadLocal<Boolean>() {
         @Override protected Boolean initialValue() {
@@ -150,7 +147,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
 
     /**
      * Gets distributed class loader. Note that
-     * {@link #p2pContext(UUID, IgniteUuid, String, DeploymentMode, Map, boolean)} must be
+     * {@link #p2pContext(UUID, IgniteUuid, String, DeploymentMode, Map)} must be
      * called from the same thread prior to using this class loader, or the
      * loading may happen for the wrong node or context.
      *
@@ -164,16 +161,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
      * Callback on method enter.
      */
     public void onEnter() {
-        if (depEnabled && !locDepOwner && !ignoreOwnership.get()
-            && !cctx.kernalContext().job().internal()) {
-            ClassLoader ldr = Thread.currentThread().getContextClassLoader();
-
-            // We mark node as deployment owner if accessing cache not from p2p deployed job
-            // and not from internal job.
-            if (!U.p2pLoader(ldr))
-                // If not deployment class loader, classes can be loaded from this node.
-                locDepOwner = true;
-        }
+        // No-op.
     }
 
     /**
@@ -369,10 +357,14 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
      * @param userVer User version.
      * @param mode Deployment mode.
      * @param participants Node participants.
-     * @param locDepOwner {@code True} if local deployment owner.
      */
-    public void p2pContext(UUID sndId, IgniteUuid ldrId, String userVer, DeploymentMode mode,
-        Map<UUID, IgniteUuid> participants, boolean locDepOwner) {
+    public void p2pContext(
+        UUID sndId,
+        IgniteUuid ldrId,
+        String userVer,
+        DeploymentMode mode,
+        Map<UUID, IgniteUuid> participants
+    ) {
         assert depEnabled;
 
         if (mode == PRIVATE || mode == ISOLATED) {
@@ -414,7 +406,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
         if (log.isDebugEnabled())
             log.debug("Setting p2p context [sndId=" + sndId + ", ldrId=" +  ldrId + ", userVer=" + userVer +
                 ", seqNum=" + ldrId.localId() + ", mode=" + mode + ", participants=" + participants +
-                ", locDepOwner=" + locDepOwner + ']');
+                ", locDepOwner=false]");
 
         CachedDeploymentInfo<K, V> depInfo;
 
@@ -443,18 +435,10 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
             break;
         }
 
-        Map<UUID, IgniteUuid> added = null;
-
-        if (locDepOwner)
-            added = addGlobalParticipants(sndId, ldrId, participants, locDepOwner);
-
         if (cctx.discovery().node(sndId) == null) {
             // Sender has left.
             deps.remove(ldrId, depInfo);
 
-            if (added != null)
-                added.remove(sndId);
-
             allParticipants.remove(sndId);
         }
 
@@ -464,16 +448,11 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
                     if (depInfo.removeParticipant(id))
                         deps.remove(ldrId, depInfo);
 
-                    if (added != null)
-                        added.remove(id);
 
                     allParticipants.remove(id);
                 }
             }
         }
-
-        if (added != null && !added.isEmpty())
-            cctx.gridDeploy().addCacheParticipants(allParticipants, added);
     }
 
     /**
@@ -690,8 +669,6 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
                 if (locDep0 != null) {
                     // Will copy sequence number to bean.
                     dep = new GridDeploymentInfoBean(locDep0);
-
-                    dep.localDeploymentOwner(locDepOwner);
                 }
             }
 
@@ -709,9 +686,6 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
     @Nullable public GridDeploymentInfoBean globalDeploymentInfo() {
         assert depEnabled;
 
-        if (locDepOwner)
-            return null;
-
         // Do not return info if mode is CONTINUOUS.
         // In this case deployment info will be set by GridCacheMessage.prepareObject().
         if (cctx.gridConfig().getDeploymentMode() == CONTINUOUS)
@@ -729,8 +703,12 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
                 for (UUID id : participants.keySet()) {
                     if (cctx.discovery().node(id) != null) {
                         // At least 1 participant is still in the grid.
-                        return new GridDeploymentInfoBean(d.loaderId(), d.userVersion(), d.mode(),
-                            participants, locDepOwner);
+                        return new GridDeploymentInfoBean(
+                            d.loaderId(),
+                            d.userVersion(),
+                            d.mode(),
+                            participants
+                        );
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/73e15783/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 6529795..e6e40e2 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
@@ -1510,8 +1510,13 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             assert depEnabled : "Received deployment info while peer class loading is disabled [nodeId=" + nodeId +
                 ", msg=" + cacheMsg + ']';
 
-            cctx.deploy().p2pContext(nodeId, bean.classLoaderId(), bean.userVersion(),
-                bean.deployMode(), bean.participants(), bean.localDeploymentOwner());
+            cctx.deploy().p2pContext(
+                nodeId,
+                bean.classLoaderId(),
+                bean.userVersion(),
+                bean.deployMode(),
+                bean.participants()
+            );
 
             if (log.isDebugEnabled())
                 log.debug("Set P2P context [senderId=" + nodeId + ", msg=" + cacheMsg + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/73e15783/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index 2b696a5..59b2a68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -726,8 +726,13 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
                 GridDeploymentInfo depInfo = e.deployInfo();
 
                 if (depInfo != null) {
-                    depMgr.p2pContext(nodeId, depInfo.classLoaderId(), depInfo.userVersion(), depInfo.deployMode(),
-                        depInfo.participants(), depInfo.localDeploymentOwner());
+                    depMgr.p2pContext(
+                        nodeId,
+                        depInfo.classLoaderId(),
+                        depInfo.userVersion(),
+                        depInfo.deployMode(),
+                        depInfo.participants()
+                    );
                 }
             }
 


[10/50] [abbrv] ignite git commit: IGNITE-6304 Fixed script execution after throttling interval. (cherry picked from commit d1f178f)

Posted by yz...@apache.org.
IGNITE-6304 Fixed script execution after throttling interval.
(cherry picked from commit d1f178f)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 668e29fa2766ad5f0fda829e027b777407d9ca4f
Parents: 6318c1a
Author: vsisko <vs...@gridgain.com>
Authored: Wed Sep 20 17:27:14 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 20 17:28:35 2017 +0700

----------------------------------------------------------------------
 .../commands/alert/VisorAlertCommand.scala      | 29 ++++++++++----------
 1 file changed, 15 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/668e29fa/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
index fb60a1e..254dc2d 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
@@ -85,7 +85,7 @@ import scala.util.control.Breaks._
  *             2) Alert condition as string.
  *             3, ...) Values of alert conditions ordered as in alert command.
  *     -i
- *         Configure alert notification minimal throttling interval in seconds. Default is 60 seconds.
+ *         Configure alert notification minimal throttling interval in seconds. Default is 0 seconds.
  *
  *     -<metric>
  *         This defines a mnemonic for the metric that will be measured:
@@ -267,7 +267,7 @@ class VisorAlertCommand extends VisorConsoleCommand {
                 var script: Option[String] = None
                 val conditions = mutable.ArrayBuffer.empty[VisorAlertCondition]
                 var freq = DFLT_FREQ
-                var interval = DFLT_FREQ
+                var interval = 0L
 
                 try {
                     args.foreach(arg => {
@@ -653,25 +653,26 @@ class VisorAlertCommand extends VisorConsoleCommand {
     private def executeAlertScript(alert: VisorAlert, node: ClusterNode, values: Seq[String]) {
         val n = alert.notification
 
+        if (n.notified && System.currentTimeMillis() - n.notifiedTime < n.throttleInterval)
+            return
+
         try {
             n.script.foreach(script => {
-                if (!n.notified && (n.notifiedTime < 0 || (System.currentTimeMillis() - n.notifiedTime) > n.throttleInterval)) {
-                    val scriptFile = new File(script)
+                val scriptFile = new File(script)
 
-                    if (!scriptFile.exists())
-                        throw new FileNotFoundException("Script/executable not found: " + script)
+                if (!scriptFile.exists())
+                    throw new FileNotFoundException("Script/executable not found: " + script)
 
-                    val scriptFolder = scriptFile.getParentFile
+                val scriptFolder = scriptFile.getParentFile
 
-                    val p = Process(Seq(script, alert.name.getOrElse(alert.id), alert.conditionSpec) ++ values,
-                        Some(scriptFolder))
+                val p = Process(Seq(script, alert.name.getOrElse(alert.id), alert.conditionSpec) ++ values,
+                    Some(scriptFolder))
 
-                    p.run(ProcessLogger((fn: String) => {}))
+                p.run(ProcessLogger((fn: String) => {}))
 
-                    n.notifiedTime = System.currentTimeMillis()
+                n.notifiedTime = System.currentTimeMillis()
 
-                    n.notified = true
-                }
+                n.notified = true
             })
         }
         catch {
@@ -865,7 +866,7 @@ object VisorAlertCommand {
                 "    2) Alert condition as string.",
                 "    3, ...) Values of alert conditions ordered as in alert command."
             ),
-            "-i" -> "Configure alert notification minimal throttling interval in seconds. Default is 60 seconds.",
+            "-i" -> "Configure alert notification minimal throttling interval in seconds. Default is 0 seconds.",
             "-<metric>" -> Seq(
                 "This defines a mnemonic for the metric that will be measured:",
                 "",


[08/50] [abbrv] ignite git commit: IGNITE-6431 Web console: Fixed duplicate "Partition loss policy" field. (cherry picked from commit 2299589)

Posted by yz...@apache.org.
IGNITE-6431 Web console: Fixed duplicate "Partition loss policy" field.
(cherry picked from commit 2299589)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 433879963db63103ba597e67e703f6ab7d3c64e1
Parents: 915682b
Author: vsisko <vs...@gridgain.com>
Authored: Wed Sep 20 00:10:46 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 20 00:12:21 2017 +0700

----------------------------------------------------------------------
 .../states/configuration/caches/general.pug     | 23 ++------------------
 1 file changed, 2 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/43387996/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug b/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
index 87c67f5..50f39e6 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
@@ -45,25 +45,6 @@ include /app/helpers/jade/mixins
                 .settings-row
                     +cacheMode('Mode:', `${model}.cacheMode`, '"cacheMode"', 'PARTITIONED')
 
-                //- Since ignite 2.0
-                .settings-row(ng-if='$ctrl.available("2.0.0")')
-                    +dropdown('Partition loss policy:', model + '.partitionLossPolicy', '"partitionLossPolicy"', 'true', 'IGNORE',
-                        '[\
-                            {value: "READ_ONLY_SAFE", label: "READ_ONLY_SAFE"},\
-                            {value: "READ_ONLY_ALL", label: "READ_ONLY_ALL"},\
-                            {value: "READ_WRITE_SAFE", label: "READ_WRITE_SAFE"},\
-                            {value: "READ_WRITE_ALL", label: "READ_WRITE_ALL"},\
-                            {value: "IGNORE", label: "IGNORE"}\
-                        ]',
-                        'Defines how cache will behave in a case when one or more partitions are lost:\
-                        <ul>\
-                            <li>READ_ONLY_SAFE - all writes to the cache will be failed with an exception, reads will only be allowed for keys in non-lost partitions</li>\
-                            <li>READ_ONLY_ALL - all writes to the cache will be failed with an exception, all reads will proceed as if all partitions were in a consistent state</li>\
-                            <li>READ_WRITE_SAFE - all reads and writes will be allowed for keys in valid partitions</li>\
-                            <li>READ_WRITE_ALL - all reads and writes will proceed as if all partitions were in a consistent state</li>\
-                            <li>IGNORE - if partition is lost, reset its state and do not clear intermediate data</li>\
-                        </ul>')
-
                 .settings-row
                     +dropdown('Atomicity:', `${model}.atomicityMode`, '"atomicityMode"', 'true', 'ATOMIC',
                         '[\
@@ -96,10 +77,10 @@ include /app/helpers/jade/mixins
                         <li>READ_ONLY_SAFE - in this mode all writes to the cache will be failed with an exception,\
                             reads will only be allowed for keys in  non-lost partitions.\
                             Reads from lost partitions will be failed with an exception.</li>\
-                        <li>READ_ONLY_ALL - in this mode фll writes to the cache will be failed with an exception.\
+                        <li>READ_ONLY_ALL - in this mode all writes to the cache will be failed with an exception.\
                             All reads will proceed as if all partitions were in a consistent state.\
                             The result of reading from a lost partition is undefined and may be different on different nodes in the cluster.</li>\
-                        <li>READ_WRITE_SAFE - in this mode Aall reads and writes will be allowed for keys in valid partitions.\
+                        <li>READ_WRITE_SAFE - in this mode all reads and writes will be allowed for keys in valid partitions.\
                             All reads and writes for keys in lost partitions will be failed with an exception.</li>\
                         <li>READ_WRITE_ALL - in this mode all reads and writes will proceed as if all partitions were in a consistent state.\
                             The result of reading from a lost partition is undefined and may be different on different nodes in the cluster.</li>\


[39/50] [abbrv] ignite git commit: IGNITE-6355 Calculating cache size during cache stop sporadically fails with ClusterGroupEmptyCheckedException - Fixes #2647.

Posted by yz...@apache.org.
IGNITE-6355 Calculating cache size during cache stop sporadically fails with ClusterGroupEmptyCheckedException - Fixes #2647.

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

(cherry picked from commit 0dd9755)

(cherry picked from commit 297ed38)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 9f5c9e385e258c48ce29ca4e9cec222f737eb910
Parents: 7daefd2
Author: Ivan Rakov <iv...@gmail.com>
Authored: Mon Sep 18 16:58:10 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:29:09 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java         | 17 ++++++-----------
 .../processors/cache/IgniteCacheProxyImpl.java     |  8 ++++----
 2 files changed, 10 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9f5c9e38/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 382b94d..40f2b49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3730,20 +3730,15 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         ClusterGroup grp = modes.near ? cluster.forCacheNodes(name(), true, true, false) : cluster.forDataNodes(name());
 
-        Collection<ClusterNode> nodes = grp.nodes();
+        Collection<ClusterNode> nodes = new ArrayList<>(grp.nodes());
 
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0);
 
         ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
-        try {
-            return ctx.kernalContext().task().execute(
-                new SizeTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), null);
-        }
-        catch (ClusterGroupEmptyException e) {
-            return new GridFinishedFuture<>(0);
-        }
+        return ctx.kernalContext().task().execute(
+            new SizeTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), null);
     }
 
     /** {@inheritDoc} */
@@ -3756,7 +3751,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         ClusterGroup grp = modes.near ? cluster.forCacheNodes(name(), true, true, false) : cluster.forDataNodes(name());
 
-        Collection<ClusterNode> nodes = grp.nodes();
+        Collection<ClusterNode> nodes = new ArrayList<>(grp.nodes());
 
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0L);
@@ -3779,13 +3774,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         ClusterGroup grp = cluster.forDataNodes(name());
 
-        Collection<ClusterNode> nodes = grp.forPredicate(new IgnitePredicate<ClusterNode>() {
+        Collection<ClusterNode> nodes = new ArrayList<>(grp.forPredicate(new IgnitePredicate<ClusterNode>() {
             /** {@inheritDoc} */
             @Override public boolean apply(ClusterNode clusterNode) {
                 return ((modes.primary && aff.primaryByPartition(clusterNode, part, topVer)) ||
                         (modes.backup && aff.backupByPartition(clusterNode, part, topVer)));
             }
-        }).nodes();
+        }).nodes());
 
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0L);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9f5c9e38/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
index 54fcafa..337c1bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
@@ -1662,10 +1662,10 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
     private RuntimeException cacheException(Exception e) {
         GridFutureAdapter<Void> restartFut = this.restartFut.get();
 
-        if (restartFut != null && !restartFut.isDone()) {
+        if (restartFut != null) {
             if (X.hasCause(e, CacheStoppedException.class) || X.hasSuppressed(e, CacheStoppedException.class))
                 throw new IgniteCacheRestartingException(new IgniteFutureImpl<>(restartFut), "Cache is restarting: " +
-                        ctx.name());
+                        ctx.name(), e);
         }
 
         if (e instanceof IgniteCheckedException)
@@ -1816,9 +1816,9 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
         this.ctx = ctx;
         this.delegate = delegate;
 
-        restartFut.onDone();
-
         this.restartFut.compareAndSet(restartFut, null);
+
+        restartFut.onDone();
     }
 
     /** {@inheritDoc} */


[35/50] [abbrv] ignite git commit: IGNITE-5658 Optimizations for data streamer

Posted by yz...@apache.org.
IGNITE-5658 Optimizations for data streamer

(cherry picked from commit aa81dd1)

(cherry picked from commit 1ad4f14)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 212603e1a969b9320f40207cd9233bed9152b3e4
Parents: 98afbfd
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Wed Aug 9 19:29:39 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:28:11 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteDataStreamer.java   |  26 +-
 .../apache/ignite/IgniteSystemProperties.java   |  12 +-
 .../ignite/internal/GridKernalContext.java      |   2 +-
 .../ignite/internal/GridKernalContextImpl.java  |   6 +-
 .../apache/ignite/internal/IgniteKernal.java    |   5 +-
 .../ignite/internal/IgniteNodeAttributes.java   |   3 +
 .../org/apache/ignite/internal/IgnitionEx.java  |  23 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |   5 +-
 .../managers/communication/GridIoManager.java   |   6 +
 .../managers/communication/GridIoMessage.java   |   3 +
 .../cache/persistence/freelist/PagesList.java   |  67 +++--
 .../wal/reader/StandaloneGridKernalContext.java |   2 +-
 .../datastreamer/DataStreamerImpl.java          | 281 ++++++++++++-------
 .../datastreamer/DataStreamerRequest.java       |  59 +++-
 .../ignite/internal/util/StripedExecutor.java   |  89 +++++-
 .../org/apache/ignite/thread/IgniteThread.java  |  10 +
 .../DataStreamProcessorSelfTest.java            |  14 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |   3 +-
 18 files changed, 424 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
index e2473dc..b1f5851 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
@@ -20,6 +20,7 @@ package org.apache.ignite;
 import java.util.Collection;
 import java.util.Map;
 import javax.cache.CacheException;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.stream.StreamReceiver;
@@ -72,7 +73,7 @@ import org.jetbrains.annotations.Nullable;
  *      this setting limits maximum allowed number of parallel buffered stream messages that
  *      are being processed on remote nodes. If this number is exceeded, then
  *      {@link #addData(Object, Object)} method will block to control memory utilization.
- *      Default is defined by {@link #DFLT_MAX_PARALLEL_OPS} value.
+ *      Default is equal to CPU count on remote node multiply by {@link #DFLT_PARALLEL_OPS_MULTIPLIER}.
  *  </li>
  *  <li>
  *      {@link #autoFlushFrequency(long)} - automatic flush frequency in milliseconds. Essentially,
@@ -100,11 +101,23 @@ import org.jetbrains.annotations.Nullable;
  * </ul>
  */
 public interface IgniteDataStreamer<K, V> extends AutoCloseable {
-    /** Default max concurrent put operations count. */
+    /**
+     * Default max concurrent put operations count.
+     * @deprecated Is not used anymore.
+     */
+    @Deprecated
     public static final int DFLT_MAX_PARALLEL_OPS = 16;
 
-    /** Default per node buffer size. */
-    public static final int DFLT_PER_NODE_BUFFER_SIZE = 1024;
+    /**
+     * Default multiplier for data streamer pool size to get concurrent batches count for each remote node.
+     *
+     * @see IgniteConfiguration#getDataStreamerThreadPoolSize()
+     * @see #perNodeParallelOperations()
+     */
+    public static final int DFLT_PARALLEL_OPS_MULTIPLIER = 8;
+
+    /** Default operations batch size to sent to remote node for loading. */
+    public static final int DFLT_PER_NODE_BUFFER_SIZE = 512;
 
     /** Default timeout for streamer's operations. */
     public static final long DFLT_UNLIMIT_TIMEOUT = -1;
@@ -203,9 +216,11 @@ public interface IgniteDataStreamer<K, V> extends AutoCloseable {
      * <p>
      * This method should be called prior to {@link #addData(Object, Object)} call.
      * <p>
-     * If not provided, default value is {@link #DFLT_MAX_PARALLEL_OPS}.
+     * If not provided, default value is calculated as follows
+     * {@link #DFLT_PARALLEL_OPS_MULTIPLIER} * {@code DATA_STREAMER_POOL_SIZE_ON_REMOTE_NODE}.
      *
      * @param parallelOps Maximum number of parallel stream operations for a single node.
+     * @see IgniteConfiguration#getDataStreamerThreadPoolSize()
      */
     public void perNodeParallelOperations(int parallelOps);
 
@@ -450,5 +465,4 @@ public interface IgniteDataStreamer<K, V> extends AutoCloseable {
      */
     @Override public void close() throws CacheException, IgniteInterruptedException,
         IgniteDataStreamerTimeoutException;
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/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 f627e24..ec79026 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.Properties;
 import javax.net.ssl.HostnameVerifier;
 import org.apache.ignite.cluster.ClusterGroup;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
@@ -707,9 +706,12 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_ENABLE_FORCIBLE_NODE_KILL = "IGNITE_ENABLE_FORCIBLE_NODE_KILL";
 
     /**
-     * If this property is set, then Ignite will use Async File IO factory by default.
+     * Tasks stealing will be started if tasks queue size per data-streamer thread exceeds this threshold.
+     * <p>
+     * Default value is {@code 4}.
      */
-    public static final String IGNITE_USE_ASYNC_FILE_IO_FACTORY = "IGNITE_USE_ASYNC_FILE_IO_FACTORY";
+    public static final String IGNITE_DATA_STREAMING_EXECUTOR_SERVICE_TASKS_STEALING_THRESHOLD =
+            "IGNITE_DATA_STREAMING_EXECUTOR_SERVICE_TASKS_STEALING_THRESHOLD";
 
     /**
      * If the property is set {@link org.apache.ignite.internal.pagemem.wal.record.TxRecord} records
@@ -719,10 +721,6 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_WAL_LOG_TX_RECORDS = "IGNITE_WAL_LOG_TX_RECORDS";
 
-    /** If this property is set, {@link PersistentStoreConfiguration#writeThrottlingEnabled} will be overridden to true
-     * independent of initial value in configuration. */
-    public static final String IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED = "IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED";
-
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 93ae465..99c7cce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -531,7 +531,7 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      *
      * @return Thread pool implementation to be used for data stream messages.
      */
-    public ExecutorService getDataStreamerExecutorService();
+    public StripedExecutor getDataStreamerExecutorService();
 
     /**
      * Should return an instance of fully configured thread pool to be used for

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 0c80eae..07e5970 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -316,7 +316,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
-    private ExecutorService dataStreamExecSvc;
+    private StripedExecutor dataStreamExecSvc;
 
     /** */
     @GridToStringExclude
@@ -422,7 +422,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         ExecutorService p2pExecSvc,
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
-        ExecutorService dataStreamExecSvc,
+        StripedExecutor dataStreamExecSvc,
         ExecutorService restExecSvc,
         ExecutorService affExecSvc,
         @Nullable ExecutorService idxExecSvc,
@@ -975,7 +975,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public ExecutorService getDataStreamerExecutorService() {
+    @Override public StripedExecutor getDataStreamerExecutorService() {
         return dataStreamExecSvc;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/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 b7430da..3ed6447 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
@@ -213,6 +213,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BUILD_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DATA_STREAMER_POOL_SIZE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS;
@@ -733,7 +734,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService p2pExecSvc,
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
-        ExecutorService dataStreamExecSvc,
+        StripedExecutor dataStreamExecSvc,
         ExecutorService restExecSvc,
         ExecutorService affExecSvc,
         @Nullable ExecutorService idxExecSvc,
@@ -1457,6 +1458,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      */
     @SuppressWarnings({"SuspiciousMethodCalls", "unchecked", "TypeMayBeWeakened"})
     private void fillNodeAttributes(boolean notifyEnabled) throws IgniteCheckedException {
+        ctx.addNodeAttribute(ATTR_DATA_STREAMER_POOL_SIZE, configuration().getDataStreamerThreadPoolSize());
+
         final String[] incProps = cfg.getIncludeProperties();
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index e4ed44a..024f339 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -180,6 +180,9 @@ public final class IgniteNodeAttributes {
     /** Ignite security compatibility mode. */
     public static final String ATTR_SECURITY_COMPATIBILITY_MODE = ATTR_PREFIX + ".security.compatibility.enabled";
 
+    /** */
+    public static final String ATTR_DATA_STREAMER_POOL_SIZE = ATTR_PREFIX + ".data.streamer.pool.size";
+
     /** Memory configuration. */
     public static final String ATTR_MEMORY_CONFIG = ATTR_PREFIX + ".memory";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/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 23baeb3..07a5c43 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
@@ -1508,7 +1508,7 @@ public class IgnitionEx {
         private ThreadPoolExecutor igfsExecSvc;
 
         /** Data streamer executor service. */
-        private ThreadPoolExecutor dataStreamerExecSvc;
+        private StripedExecutor dataStreamerExecSvc;
 
         /** REST requests executor service. */
         private ThreadPoolExecutor restExecSvc;
@@ -1728,7 +1728,11 @@ public class IgnitionEx {
 
             validateThreadPoolSize(cfg.getStripedPoolSize(), "stripedPool");
 
-            stripedExecSvc = new StripedExecutor(cfg.getStripedPoolSize(), cfg.getIgniteInstanceName(), "sys", log);
+            stripedExecSvc = new StripedExecutor(
+                cfg.getStripedPoolSize(),
+                cfg.getIgniteInstanceName(),
+                "sys",
+                log);
 
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
@@ -1763,17 +1767,12 @@ public class IgnitionEx {
 
             p2pExecSvc.allowCoreThreadTimeOut(true);
 
-            // Note that we do not pre-start threads here as this pool may not be needed.
-            dataStreamerExecSvc = new IgniteThreadPoolExecutor(
-                "data-streamer",
-                cfg.getIgniteInstanceName(),
-                cfg.getDataStreamerThreadPoolSize(),
+            dataStreamerExecSvc = new StripedExecutor(
                 cfg.getDataStreamerThreadPoolSize(),
-                DFLT_THREAD_KEEP_ALIVE_TIME,
-                new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.DATA_STREAMER_POOL);
-
-            dataStreamerExecSvc.allowCoreThreadTimeOut(true);
+                cfg.getIgniteInstanceName(),
+                "data-streamer",
+                log,
+                true);
 
             // Note that we do not pre-start threads here as igfs pool may not be needed.
             validateThreadPoolSize(cfg.getIgfsThreadPoolSize(), "IGFS");

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/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 6c8fc0b..6b8371d 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
@@ -199,8 +199,9 @@ public class JdbcConnection implements Connection {
         streamFlushTimeout = Long.parseLong(props.getProperty(PROP_STREAMING_FLUSH_FREQ, "0"));
         streamNodeBufSize = Integer.parseInt(props.getProperty(PROP_STREAMING_PER_NODE_BUF_SIZE,
             String.valueOf(IgniteDataStreamer.DFLT_PER_NODE_BUFFER_SIZE)));
-        streamNodeParOps = Integer.parseInt(props.getProperty(PROP_STREAMING_PER_NODE_PAR_OPS,
-            String.valueOf(IgniteDataStreamer.DFLT_MAX_PARALLEL_OPS)));
+        // If value is zero, server data-streamer pool size multiplied
+        // by IgniteDataStreamer.DFLT_PARALLEL_OPS_MULTIPLIER will be used
+        streamNodeParOps = Integer.parseInt(props.getProperty(PROP_STREAMING_PER_NODE_PAR_OPS, "0"));
 
         String nodeIdProp = props.getProperty(PROP_NODE_ID);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index bb36b26..2005032 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -1125,6 +1125,12 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             return;
         }
 
+        if (plc == GridIoPolicy.DATA_STREAMER_POOL && msg.partition() != GridIoMessage.STRIPE_DISABLED_PART) {
+            ctx.getDataStreamerExecutorService().execute(msg.partition(), c);
+
+            return;
+        }
+
         if (msg.topicOrdinal() == TOPIC_IO_TEST.ordinal()) {
             IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg.message();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index dccd336..fe61aec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.ignite.internal.ExecutorAwareMessage;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -336,6 +337,8 @@ public class GridIoMessage implements Message {
     public int partition() {
         if (msg instanceof GridCacheMessage)
             return ((GridCacheMessage)msg).partition();
+        if (msg instanceof DataStreamerRequest)
+            return ((DataStreamerRequest)msg).partition();
         else
             return STRIPE_DISABLED_PART;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
index 39a6865..6c355f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
@@ -47,6 +47,7 @@ import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
 import static java.lang.Boolean.FALSE;
@@ -66,7 +67,7 @@ public abstract class PagesList extends DataStructure {
     /** */
     private static final int MAX_STRIPES_PER_BUCKET =
         IgniteSystemProperties.getInteger("IGNITE_PAGES_LIST_STRIPES_PER_BUCKET",
-            Math.min(8, Runtime.getRuntime().availableProcessors() * 2));
+            Math.max(8, Runtime.getRuntime().availableProcessors()));
 
     /** */
     protected final AtomicLong[] bucketsSize;
@@ -507,6 +508,21 @@ public abstract class PagesList extends DataStructure {
      * @throws IgniteCheckedException If failed.
      */
     private Stripe getPageForPut(int bucket) throws IgniteCheckedException {
+        // Striped pool optimization.
+        int stripeIdx; IgniteThread igniteThread = IgniteThread.current();
+
+        if (igniteThread != null && (stripeIdx = igniteThread.stripe()) != -1) {
+            Stripe[] tails = getBucket(bucket);
+
+            while (tails == null || stripeIdx >= tails.length) {
+                addStripe(bucket, true);
+
+                tails = getBucket(bucket);
+            }
+
+            return tails[stripeIdx];
+        }
+
         Stripe[] tails = getBucket(bucket);
 
         if (tails == null)
@@ -607,12 +623,8 @@ public abstract class PagesList extends DataStructure {
             try {
                 long tailAddr = writeLockPage(tailId, tailPage, bucket, lockAttempt++); // Explicit check.
 
-                if (tailAddr == 0L) {
-                    if (isReuseBucket(bucket) && lockAttempt == TRY_LOCK_ATTEMPTS)
-                        addStripeForReuseBucket(bucket);
-
+                if (tailAddr == 0L)
                     continue;
-                }
 
                 assert PageIO.getPageId(tailAddr) == tailId : "pageId = " + PageIO.getPageId(tailAddr) + ", tailId = " + tailId;
                 assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE;
@@ -912,13 +924,26 @@ public abstract class PagesList extends DataStructure {
      * @param bucket Bucket index.
      * @return Page for take.
      */
-    private Stripe getPageForTake(int bucket) {
+    private Stripe getPageForTake(int bucket) throws IgniteCheckedException {
         Stripe[] tails = getBucket(bucket);
 
         if (tails == null || bucketsSize[bucket].get() == 0)
             return null;
 
         int len = tails.length;
+
+        // Striped pool optimization.
+        int stripeIdx; IgniteThread igniteThread = IgniteThread.current();
+
+        if (igniteThread != null && (stripeIdx = igniteThread.stripe()) != -1) {
+            if (stripeIdx >= len)
+                return null;
+
+            Stripe stripe = tails[stripeIdx];
+
+            return stripe.empty ? null : stripe;
+        }
+
         int init = randomInt(len);
         int cur = init;
 
@@ -943,6 +968,12 @@ public abstract class PagesList extends DataStructure {
      */
     private long writeLockPage(long pageId, long page, int bucket, int lockAttempt)
         throws IgniteCheckedException {
+        // Striped pool optimization.
+        IgniteThread igniteThread = IgniteThread.current();
+
+        if (igniteThread != null && igniteThread.stripe() != -1)
+            return writeLock(pageId, page);
+
         long pageAddr = tryWriteLock(pageId, page);
 
         if (pageAddr != 0L)
@@ -952,8 +983,7 @@ public abstract class PagesList extends DataStructure {
             Stripe[] stripes = getBucket(bucket);
 
             if (stripes == null || stripes.length < MAX_STRIPES_PER_BUCKET) {
-                if (!isReuseBucket(bucket))
-                    addStripe(bucket, true);
+                addStripe(bucket, !isReuseBucket(bucket));
 
                 return 0L;
             }
@@ -963,19 +993,6 @@ public abstract class PagesList extends DataStructure {
     }
 
     /**
-     * @param bucket Bucket.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void addStripeForReuseBucket(int bucket) throws IgniteCheckedException {
-        assert isReuseBucket(bucket);
-
-        Stripe[] stripes = getBucket(bucket);
-
-        if (stripes == null || stripes.length < MAX_STRIPES_PER_BUCKET)
-            addStripe(bucket, false);
-    }
-
-    /**
      * @param bucket Bucket index.
      * @param initIoVers Optional IO to initialize page.
      * @return Removed page ID.
@@ -994,12 +1011,8 @@ public abstract class PagesList extends DataStructure {
             try {
                 long tailAddr = writeLockPage(tailId, tailPage, bucket, lockAttempt++); // Explicit check.
 
-                if (tailAddr == 0L) {
-                    if (isReuseBucket(bucket) && lockAttempt == TRY_LOCK_ATTEMPTS)
-                        addStripeForReuseBucket(bucket);
-
+                if (tailAddr == 0L)
                     continue;
-                }
 
                 if (stripe.empty) {
                     // Another thread took the last page.

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 9dfd338..07be8b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -513,7 +513,7 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     }
 
     /** {@inheritDoc} */
-    @Override public ExecutorService getDataStreamerExecutorService() {
+    @Override public StripedExecutor getDataStreamerExecutorService() {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 6681710..1869dcf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.datastreamer;
 
+import java.lang.reflect.Array;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -60,8 +61,10 @@ import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
@@ -152,7 +155,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     private int bufSize = DFLT_PER_NODE_BUFFER_SIZE;
 
     /** */
-    private int parallelOps = DFLT_MAX_PARALLEL_OPS;
+    private int parallelOps;
 
     /** */
     private long timeout = DFLT_UNLIMIT_TIMEOUT;
@@ -794,6 +797,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                             initPda = false;
                         }
 
+                        if (key.partition() == -1)
+                            key.partition(cctx.affinity().partition(key, false));
+
                         nodes = nodes(key, topVer, cctx);
                     }
                     catch (IgniteCheckedException e) {
@@ -935,10 +941,10 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                         }
                     };
 
-                    final GridFutureAdapter<?> f;
+                    final List<GridFutureAdapter<?>> futs;
 
                     try {
-                        f = buf.update(entriesForNode, topVer, lsnr, remap);
+                        futs = buf.update(entriesForNode, topVer, lsnr, remap);
                     }
                     catch (IgniteInterruptedCheckedException e1) {
                         resFut.onDone(e1);
@@ -954,9 +960,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                                 @Override public void run() {
                                     buf0.onNodeLeft();
 
-                                    if (f != null)
-                                        f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
-                                            "(node has left): " + nodeId));
+                                    if (futs != null) {
+                                        Throwable ex = new ClusterTopologyCheckedException(
+                                                "Failed to wait for request completion (node has left): " + nodeId);
+
+                                        for (int i = 0; i < futs.size(); i++)
+                                            futs.get(i).onDone(ex);
+                                    }
                                 }
                             }, ctx.discovery().topologyVersion(), false);
                         }
@@ -1314,11 +1324,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         private final Collection<IgniteInternalFuture<Object>> locFuts;
 
         /** Buffered entries. */
-        private List<DataStreamerEntry> entries;
-
-        /** */
-        @GridToStringExclude
-        private GridFutureAdapter<Object> curFut;
+        private final PerStripeBuffer[] stripes;
 
         /** Local node flag. */
         private final boolean isLocNode;
@@ -1332,16 +1338,17 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         /** */
         private final Semaphore sem;
 
-        /** Batch topology. */
-        private AffinityTopologyVersion batchTopVer;
+        /** */
+        private final int perNodeParallelOps;
 
         /** Closure to signal on task finish. */
         @GridToStringExclude
-        private final IgniteInClosure<IgniteInternalFuture<Object>> signalC = new IgniteInClosure<IgniteInternalFuture<Object>>() {
-            @Override public void apply(IgniteInternalFuture<Object> t) {
-                signalTaskFinished(t);
-            }
-        };
+        private final IgniteInClosure<IgniteInternalFuture<Object>> signalC =
+            new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                @Override public void apply(IgniteInternalFuture<Object> t) {
+                    signalTaskFinished(t);
+                }
+            };
 
         /**
          * @param node Node.
@@ -1357,24 +1364,19 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             // Cache local node flag.
             isLocNode = node.equals(ctx.discovery().localNode());
 
-            entries = newEntries();
-            curFut = new GridFutureAdapter<>();
-            curFut.listen(signalC);
+            Integer attrStreamerPoolSize = node.attribute(IgniteNodeAttributes.ATTR_DATA_STREAMER_POOL_SIZE);
 
-            sem = new Semaphore(parallelOps);
-        }
+            int streamerPoolSize = attrStreamerPoolSize != null ? attrStreamerPoolSize : node.metrics().getTotalCpus();
 
-        /**
-         * @param remap Remapping flag.
-         */
-        private void renewBatch(boolean remap) {
-            entries = newEntries();
-            curFut = new GridFutureAdapter<>();
+            perNodeParallelOps = parallelOps != 0 ? parallelOps :
+                streamerPoolSize * IgniteDataStreamer.DFLT_PARALLEL_OPS_MULTIPLIER;
 
-            batchTopVer = null;
+            sem = new Semaphore(perNodeParallelOps);
 
-            if (!remap)
-                curFut.listen(signalC);
+            stripes = (PerStripeBuffer[])Array.newInstance(PerStripeBuffer.class, streamerPoolSize);
+
+            for (int i = 0; i < stripes.length; i++)
+                stripes[i] = new PerStripeBuffer(i, signalC);
         }
 
         /**
@@ -1385,61 +1387,69 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
          * @return Future for operation.
          * @throws IgniteInterruptedCheckedException If failed.
          */
-        @Nullable GridFutureAdapter<?> update(Iterable<DataStreamerEntry> newEntries,
+        @Nullable List<GridFutureAdapter<?>> update(
+            Iterable<DataStreamerEntry> newEntries,
             AffinityTopologyVersion topVer,
             IgniteInClosure<IgniteInternalFuture<?>> lsnr,
-            boolean remap) throws IgniteInterruptedCheckedException {
-            List<DataStreamerEntry> entries0 = null;
+            boolean remap
+        ) throws IgniteInterruptedCheckedException {
+            List<GridFutureAdapter<?>> res = null;
 
-            GridFutureAdapter<Object> curFut0;
+            for (DataStreamerEntry entry : newEntries) {
+                List<DataStreamerEntry> entries0 = null;
+                AffinityTopologyVersion curBatchTopVer;
+
+                // Init buffer.
+                int part = entry.getKey().partition();
 
-            AffinityTopologyVersion curBatchTopVer;
+                GridFutureAdapter<Object> curFut0;
+                PerStripeBuffer b = stripes[part % stripes.length];
 
-            synchronized (this) {
-                curFut0 = curFut;
+                synchronized (b) {
+                    curFut0 = b.curFut;
 
-                curFut0.listen(lsnr);
+                    // Listener should be added only once per whole entries collection.
+                    // Should we simplify the model and get rid of all futures?
+                    curFut0.listen(lsnr);
 
-                if (batchTopVer == null)
-                    batchTopVer = topVer;
+                    if (b.batchTopVer == null)
+                        b.batchTopVer = topVer;
 
-                curBatchTopVer = batchTopVer;
+                    curBatchTopVer = b.batchTopVer;
 
-                for (DataStreamerEntry entry : newEntries)
-                    entries.add(entry);
+                    b.entries.add(entry);
 
-                if (entries.size() >= bufSize) {
-                    entries0 = entries;
+                    if (b.entries.size() >= bufSize) {
+                        entries0 = b.entries;
 
-                    renewBatch(remap);
+                        b.renewBatch(remap);
+                    }
                 }
-            }
 
-            if (!allowOverwrite() && !topVer.equals(curBatchTopVer)) {
-                renewBatch(remap);
+                if (res == null)
+                    res = new ArrayList<>();
 
-                curFut0.onDone(null, new IgniteCheckedException("Topology changed during batch preparation." +
-                    "[batchTopVer=" + curBatchTopVer + ", topVer=" + topVer + "]"));
-            }
-            else if (entries0 != null) {
-                submit(entries0, curBatchTopVer, curFut0, remap);
-
-                if (cancelled)
-                    curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " +
-                        DataStreamerImpl.this));
-                else if (ctx.clientDisconnected())
-                    curFut0.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
-                        "Client node disconnected."));
-            }
+                res.add(curFut0);
 
-            return curFut0;
-        }
+                if (!allowOverwrite() && !topVer.equals(curBatchTopVer)) {
+                    b.renewBatch(remap);
 
-        /**
-         * @return Fresh collection with some space for outgrowth.
-         */
-        private List<DataStreamerEntry> newEntries() {
-            return new ArrayList<>((int)(bufSize * 1.2));
+                    curFut0.onDone(null, new IgniteCheckedException("Topology changed during batch preparation." +
+                        "[batchTopVer=" + curBatchTopVer + ", topVer=" + topVer + "]"));
+                }
+                else if (entries0 != null) {
+                    submit(entries0, curBatchTopVer, curFut0, remap, b.partId);
+
+                    if (cancelled)
+                        curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " +
+                            DataStreamerImpl.this));
+                    else if (ctx.clientDisconnected())
+                        curFut0.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                            "Client node disconnected."));
+                }
+            }
+
+            return res;
         }
 
         /**
@@ -1447,24 +1457,26 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
          * @throws IgniteInterruptedCheckedException If thread has been interrupted.
          */
         @Nullable IgniteInternalFuture<?> flush() throws IgniteInterruptedCheckedException {
-            List<DataStreamerEntry> entries0 = null;
-            GridFutureAdapter<Object> curFut0 = null;
-
             acquireRemapSemaphore();
 
-            synchronized (this) {
-                if (!entries.isEmpty()) {
-                    entries0 = entries;
-                    curFut0 = curFut;
+            for (PerStripeBuffer b : stripes) {
+                AffinityTopologyVersion batchTopVer = null;
+                List<DataStreamerEntry> entries0 = null;
+                GridFutureAdapter<Object> curFut0 = null;
+
+                synchronized (b) {
+                    if (!b.entries.isEmpty()) {
+                        entries0 = b.entries;
+                        curFut0 = b.curFut;
+                        batchTopVer = b.batchTopVer;
 
-                    entries = newEntries();
-                    curFut = new GridFutureAdapter<>();
-                    curFut.listen(signalC);
+                        b.renewBatch(false);
+                    }
                 }
-            }
 
-            if (entries0 != null)
-                submit(entries0, batchTopVer, curFut0, false);
+                if (entries0 != null)
+                    submit(entries0, batchTopVer, curFut0, false, b.partId);
+            }
 
             // Create compound future for this flush.
             GridCompoundFuture<Object, Object> res = null;
@@ -1618,13 +1630,16 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
          * @param topVer Topology version.
          * @param curFut Current future.
          * @param remap Remapping flag.
+         * @param partId Partition ID.
          * @throws IgniteInterruptedCheckedException If interrupted.
          */
-        private void submit(final Collection<DataStreamerEntry> entries,
+        private void submit(
+            final Collection<DataStreamerEntry> entries,
             @Nullable AffinityTopologyVersion topVer,
             final GridFutureAdapter<Object> curFut,
-            boolean remap)
-            throws IgniteInterruptedCheckedException {
+            boolean remap,
+            int partId
+        ) throws IgniteInterruptedCheckedException {
             assert entries != null;
             assert !entries.isEmpty();
             assert curFut != null;
@@ -1685,7 +1700,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                             cache.context().deploy().onEnter();
                     }
                     catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to deploy class (request will not be sent): " + jobPda0.deployClass(), e);
+                        U.error(log, "Failed to deploy class (request will not be sent): " +
+                            jobPda0.deployClass(), e);
 
                         return;
                     }
@@ -1718,7 +1734,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     dep != null ? dep.participants() : null,
                     dep != null ? dep.classLoaderId() : null,
                     dep == null,
-                    topVer);
+                    topVer,
+                    (rcvr == ISOLATED_UPDATER) ?
+                        partId : GridIoMessage.STRIPE_DISABLED_PART);
 
                 try {
                     ctx.io().sendToGridTopic(node, TOPIC_DATASTREAM, req, plc);
@@ -1767,11 +1785,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             // Make sure to complete current future.
             GridFutureAdapter<Object> curFut0;
 
-            synchronized (this) {
-                curFut0 = curFut;
-            }
+            for (PerStripeBuffer b : stripes) {
+                synchronized (b) {
+                    curFut0 = b.curFut;
+                }
 
-            curFut0.onDone(e);
+                curFut0.onDone(e);
+            }
         }
 
         /**
@@ -1845,10 +1865,14 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            int size;
+            int size = 0;
 
-            synchronized (this) {
-                size = entries.size();
+            for (int i = 0; i < stripes.length; i++) {
+                PerStripeBuffer b = stripes[i];
+
+                synchronized (b) {
+                    size += b.entries.size();
+                }
             }
 
             return S.toString(Buffer.class, this,
@@ -1937,8 +1961,10 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public void receive(IgniteCache<KeyCacheObject, CacheObject> cache,
-            Collection<Map.Entry<KeyCacheObject, CacheObject>> entries) {
+        @Override public void receive(
+            IgniteCache<KeyCacheObject, CacheObject> cache,
+            Collection<Map.Entry<KeyCacheObject, CacheObject>> entries
+        ) {
             IgniteCacheProxy<KeyCacheObject, CacheObject> proxy = (IgniteCacheProxy<KeyCacheObject, CacheObject>)cache;
 
             GridCacheAdapter<KeyCacheObject, CacheObject> internalCache = proxy.context().cache();
@@ -2097,4 +2123,63 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             return S.toString(KeyCacheObjectWrapper.class, this);
         }
     }
+
+    /**
+     *
+     */
+    private class PerStripeBuffer {
+        /** */
+        private final int partId;
+
+        /** */
+        private List<DataStreamerEntry> entries;
+
+        /** */
+        private GridFutureAdapter<Object> curFut;
+
+        /** Batch topology. */
+        private AffinityTopologyVersion batchTopVer;
+
+        /** */
+        private final IgniteInClosure<? super IgniteInternalFuture<Object>> signalC;
+
+        /**
+         * @param partId Partition ID.
+         * @param c Signal closure.
+         */
+        public PerStripeBuffer(
+            int partId,
+            IgniteInClosure<? super IgniteInternalFuture<Object>> c
+        ) {
+            this.partId = partId;
+            signalC = c;
+
+            renewBatch(false);
+        }
+
+        /**
+         * @param remap Remap.
+         */
+        synchronized void renewBatch(boolean remap) {
+            entries = newEntries();
+            curFut = new GridFutureAdapter<>();
+
+            batchTopVer = null;
+
+            if (!remap)
+                curFut.listen(signalC);
+        }
+
+        /**
+         * @return Fresh collection with some space for outgrowth.
+         */
+        private List<DataStreamerEntry> newEntries() {
+            return new ArrayList<>((int)(bufSize * 1.2));
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(PerStripeBuffer.class, this, super.toString());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
index b4cbf66..f70ee9c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
@@ -90,6 +90,9 @@ public class DataStreamerRequest implements Message {
     /** Topology version. */
     private AffinityTopologyVersion topVer;
 
+    /** */
+    private int partId;
+
     /**
      * {@code Externalizable} support.
      */
@@ -113,8 +116,10 @@ public class DataStreamerRequest implements Message {
      * @param clsLdrId Class loader ID.
      * @param forceLocDep Force local deployment.
      * @param topVer Topology version.
+     * @param partId Partition ID.
      */
-    public DataStreamerRequest(long reqId,
+    public DataStreamerRequest(
+        long reqId,
         byte[] resTopicBytes,
         @Nullable String cacheName,
         byte[] updaterBytes,
@@ -128,7 +133,9 @@ public class DataStreamerRequest implements Message {
         Map<UUID, IgniteUuid> ldrParticipants,
         IgniteUuid clsLdrId,
         boolean forceLocDep,
-        @NotNull AffinityTopologyVersion topVer) {
+        @NotNull AffinityTopologyVersion topVer,
+        int partId
+    ) {
         assert topVer != null;
 
         this.reqId = reqId;
@@ -146,6 +153,7 @@ public class DataStreamerRequest implements Message {
         this.clsLdrId = clsLdrId;
         this.forceLocDep = forceLocDep;
         this.topVer = topVer;
+        this.partId = partId;
     }
 
     /**
@@ -253,6 +261,13 @@ public class DataStreamerRequest implements Message {
         return topVer;
     }
 
+    /**
+     * @return Partition ID.
+     */
+    public int partition() {
+        return partId;
+    }
+
     /** {@inheritDoc} */
     @Override public void onAckReceived() {
         // No-op.
@@ -324,42 +339,48 @@ public class DataStreamerRequest implements Message {
                 writer.incrementState();
 
             case 8:
-                if (!writer.writeLong("reqId", reqId))
+                if (!writer.writeInt("partId", partId))
                     return false;
 
                 writer.incrementState();
 
             case 9:
-                if (!writer.writeByteArray("resTopicBytes", resTopicBytes))
+                if (!writer.writeLong("reqId", reqId))
                     return false;
 
                 writer.incrementState();
 
             case 10:
-                if (!writer.writeString("sampleClsName", sampleClsName))
+                if (!writer.writeByteArray("resTopicBytes", resTopicBytes))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeBoolean("skipStore", skipStore))
+                if (!writer.writeString("sampleClsName", sampleClsName))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeBoolean("skipStore", skipStore))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeByteArray("updaterBytes", updaterBytes))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
             case 14:
+                if (!writer.writeByteArray("updaterBytes", updaterBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 15:
                 if (!writer.writeString("userVer", userVer))
                     return false;
 
@@ -447,7 +468,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 8:
-                reqId = reader.readLong("reqId");
+                partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -455,7 +476,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 9:
-                resTopicBytes = reader.readByteArray("resTopicBytes");
+                reqId = reader.readLong("reqId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -463,7 +484,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 10:
-                sampleClsName = reader.readString("sampleClsName");
+                resTopicBytes = reader.readByteArray("resTopicBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -471,7 +492,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 11:
-                skipStore = reader.readBoolean("skipStore");
+                sampleClsName = reader.readString("sampleClsName");
 
                 if (!reader.isLastRead())
                     return false;
@@ -479,7 +500,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 12:
-                topVer = reader.readMessage("topVer");
+                skipStore = reader.readBoolean("skipStore");
 
                 if (!reader.isLastRead())
                     return false;
@@ -487,7 +508,7 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 13:
-                updaterBytes = reader.readByteArray("updaterBytes");
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -495,6 +516,14 @@ public class DataStreamerRequest implements Message {
                 reader.incrementState();
 
             case 14:
+                updaterBytes = reader.readByteArray("updaterBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 15:
                 userVer = reader.readString("userVer");
 
                 if (!reader.isLastRead())
@@ -514,6 +543,6 @@ public class DataStreamerRequest implements Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 15;
+        return 16;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
index 6d5dc71..630d34c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
@@ -20,10 +20,12 @@ package org.apache.ignite.internal.util;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.List;
 import java.util.Queue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -36,6 +38,8 @@ import java.util.concurrent.locks.LockSupport;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -62,6 +66,17 @@ public class StripedExecutor implements ExecutorService {
      * @param log Logger.
      */
     public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log) {
+        this(cnt, igniteInstanceName, poolName, log, false);
+    }
+
+    /**
+     * @param cnt Count.
+     * @param igniteInstanceName Node name.
+     * @param poolName Pool name.
+     * @param log Logger.
+     * @param stealTasks {@code True} to steal tasks.
+     */
+    public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log, boolean stealTasks) {
         A.ensure(cnt > 0, "cnt > 0");
 
         boolean success = false;
@@ -76,14 +91,19 @@ public class StripedExecutor implements ExecutorService {
 
         try {
             for (int i = 0; i < cnt; i++) {
-                stripes[i] = new StripeConcurrentQueue(
+                stripes[i] = stealTasks ? new StripeConcurrentQueue(
                     igniteInstanceName,
                     poolName,
                     i,
-                    log);
+                    log, stripes) : new StripeConcurrentQueue(
+                        igniteInstanceName,
+                        poolName,
+                        i,
+                        log);
+            }
 
+            for (int i = 0; i < cnt; i++)
                 stripes[i].start();
-            }
 
             success = true;
         }
@@ -397,7 +417,7 @@ public class StripedExecutor implements ExecutorService {
         private final String poolName;
 
         /** */
-        private final int idx;
+        protected final int idx;
 
         /** */
         private final IgniteLogger log;
@@ -536,8 +556,17 @@ public class StripedExecutor implements ExecutorService {
      * Stripe.
      */
     private static class StripeConcurrentQueue extends Stripe {
+        /** */
+        private static final int IGNITE_TASKS_STEALING_THRESHOLD =
+            IgniteSystemProperties.getInteger(
+                IgniteSystemProperties.IGNITE_DATA_STREAMING_EXECUTOR_SERVICE_TASKS_STEALING_THRESHOLD, 4);
+
         /** Queue. */
-        private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+        private final Queue<Runnable> queue;
+
+        /** */
+        @GridToStringExclude
+        private final Stripe[] others;
 
         /** */
         private volatile boolean parked;
@@ -548,16 +577,37 @@ public class StripedExecutor implements ExecutorService {
          * @param idx Stripe index.
          * @param log Logger.
          */
-        public StripeConcurrentQueue(
+        StripeConcurrentQueue(
             String igniteInstanceName,
             String poolName,
             int idx,
             IgniteLogger log
         ) {
-            super(igniteInstanceName,
+            this(igniteInstanceName, poolName, idx, log, null);
+        }
+
+        /**
+         * @param igniteInstanceName Ignite instance name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        StripeConcurrentQueue(
+            String igniteInstanceName,
+            String poolName,
+            int idx,
+            IgniteLogger log,
+            Stripe[] others
+        ) {
+            super(
+                igniteInstanceName,
                 poolName,
                 idx,
                 log);
+
+            this.others = others;
+
+            this.queue = others == null ? new ConcurrentLinkedQueue<Runnable>() : new ConcurrentLinkedDeque<Runnable>();
         }
 
         /** {@inheritDoc} */
@@ -580,6 +630,24 @@ public class StripedExecutor implements ExecutorService {
                     if (r != null)
                         return r;
 
+                    if(others != null) {
+                        int len = others.length;
+                        int init = ThreadLocalRandom.current().nextInt(len);
+                        int cur = init;
+
+                        while (true) {
+                            if(cur != idx) {
+                                Deque<Runnable> queue = (Deque<Runnable>) ((StripeConcurrentQueue) others[cur]).queue;
+
+                                if(queue.size() > IGNITE_TASKS_STEALING_THRESHOLD && (r = queue.pollLast()) != null)
+                                    return r;
+                            }
+
+                            if ((cur = (cur + 1) % len) == init)
+                                break;
+                        }
+                    }
+
                     LockSupport.park();
 
                     if (Thread.interrupted())
@@ -597,6 +665,13 @@ public class StripedExecutor implements ExecutorService {
 
             if (parked)
                 LockSupport.unpark(thread);
+
+            if(others != null && queueSize() > IGNITE_TASKS_STEALING_THRESHOLD) {
+                for (Stripe other : others) {
+                    if(((StripeConcurrentQueue)other).parked)
+                        LockSupport.unpark(other.thread);
+                }
+            }
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
index 83a0384..b8a91a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
@@ -158,6 +158,16 @@ public class IgniteThread extends Thread {
     }
 
     /**
+     * @return IgniteThread or {@code null} if current thread is not an instance of IgniteThread.
+     */
+    public static IgniteThread current(){
+        Thread thread = Thread.currentThread();
+
+        return thread.getClass() == IgniteThread.class || thread instanceof IgniteThread ?
+            ((IgniteThread)thread) : null;
+    }
+
+    /**
      * Creates new thread name.
      *
      * @param num Thread number.

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
index ec5e6d0..ac89021 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
@@ -962,8 +962,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             final IgniteCache<String, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
-            IgniteDataStreamer<String, String> ldr = ignite.dataStreamer(DEFAULT_CACHE_NAME);
-            try {
+            try (IgniteDataStreamer<String, String> ldr = ignite.dataStreamer(DEFAULT_CACHE_NAME)) {
                 ldr.receiver(new StreamReceiver<String, String>() {
                     @Override public void receive(IgniteCache<String, String> cache,
                         Collection<Map.Entry<String, String>> entries) throws IgniteException {
@@ -972,6 +971,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
                         cache.put("key", threadName);
                     }
                 });
+
                 ldr.addData("key", "value");
 
                 ldr.tryFlush();
@@ -982,9 +982,6 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
                     }
                 }, 3_000);
             }
-            finally {
-                ldr.close(true);
-            }
 
             assertNotNull(cache.get("key"));
 
@@ -1011,9 +1008,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             final IgniteCache<String, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
-            IgniteDataStreamer<String, String> ldr = client.dataStreamer(DEFAULT_CACHE_NAME);
-
-            try {
+            try (IgniteDataStreamer<String, String> ldr = client.dataStreamer(DEFAULT_CACHE_NAME)) {
                 ldr.receiver(new StringStringStreamReceiver());
 
                 ldr.addData("key", "value");
@@ -1026,9 +1021,6 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
                     }
                 }, 3_000);
             }
-            finally {
-                ldr.close(true);
-            }
 
             assertNotNull(cache.get("key"));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/212603e1/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
index e72a9b4..6d3466b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
@@ -346,7 +346,8 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
                             req.participants(),
                             req.classLoaderId(),
                             req.forceLocalDeployment(),
-                            staleTop);
+                            staleTop,
+                            -1);
 
                         msg = new GridIoMessage(
                             GridTestUtils.<Byte>getFieldValue(ioMsg, "plc"),


[04/50] [abbrv] ignite git commit: IGNITE-6099: ODBC: Implemented SQLGetInfo for all info types. This closes #2689.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/718e365f/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 472eb75..777973a 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -46,7 +46,8 @@ namespace ignite
             socket(),
             connected(false),
             parser(),
-            config()
+            config(),
+            info(config)
         {
             // No-op.
         }
@@ -58,9 +59,6 @@ namespace ignite
 
         const config::ConnectionInfo& Connection::GetInfo() const
         {
-            // Connection info is constant and the same for all connections now.
-            const static config::ConnectionInfo info;
-
             return info;
         }
 


[45/50] [abbrv] ignite git commit: IGNITE-6484 writeComplete conditional wait is made uninterruptable

Posted by yz...@apache.org.
IGNITE-6484 writeComplete conditional wait is made uninterruptable


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 6452201dea49d5583e61e7fb3d33fd8c096135bd
Parents: 21de1c5
Author: Alexei Scherbakov <al...@gmail.com>
Authored: Fri Sep 22 16:36:38 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Sep 22 16:37:14 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/wal/FileWriteAheadLogManager.java     | 4 ++--
 .../java/org/apache/ignite/internal/util/IgniteUtils.java   | 9 +++++++++
 .../db/file/IgnitePdsThreadInterruptionTest.java            | 2 +-
 3 files changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6452201d/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 87069d9..9b2d948 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
@@ -1828,7 +1828,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
             try {
                 while (written < expWritten && envFailed == null)
-                    U.await(writeComplete);
+                    U.awaitQuiet(writeComplete);
             }
             finally {
                 lock.unlock();
@@ -2163,7 +2163,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
             try {
                 while (fileIO != null)
-                    U.await(nextSegment);
+                    U.awaitQuiet(nextSegment);
             }
             finally {
                 lock.unlock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/6452201d/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 13e99fb..6fb7242 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
@@ -7396,6 +7396,15 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Awaits for condition ignoring interrupts.
+     *
+     * @param cond Condition to await for.
+     */
+    public static void awaitQuiet(Condition cond) {
+        cond.awaitUninterruptibly();
+    }
+
+    /**
      * Awaits for condition.
      *
      * @param cond Condition to await for.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6452201d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
index aab569a..6f11d4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
@@ -41,7 +41,7 @@ public class IgnitePdsThreadInterruptionTest extends GridCommonAbstractTest {
     private static final int PAGE_SIZE = 1 << 12; // 4096
 
     /** */
-    public static final int THREADS_CNT = 1;
+    public static final int THREADS_CNT = 10;
 
     /**
      * Cache name.


[32/50] [abbrv] ignite git commit: IGNITE-6334 Fixed compilation after conflict

Posted by yz...@apache.org.
IGNITE-6334 Fixed compilation after conflict


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 24f904e6af7d17111bc9ec273d6555b70057e25c
Parents: 520c2e3
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Sep 22 13:12:15 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Sep 22 13:13:03 2017 +0300

----------------------------------------------------------------------
 .../persistence/pagemem/PagesWriteThrottleSmokeTest.java | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/24f904e6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
index 12a601d..f4f21ef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
@@ -20,6 +20,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
+import java.nio.file.OpenOption;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.LockSupport;
@@ -50,6 +51,10 @@ 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 static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+
 /**
  *
  */
@@ -288,12 +293,12 @@ public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public FileIO create(File file) throws IOException {
-            return create(file, "rw");
+            return create(file, CREATE, READ, WRITE);
         }
 
         /** {@inheritDoc} */
-        @Override public FileIO create(File file, String mode) throws IOException {
-            final FileIO delegate = delegateFactory.create(file, mode);
+        @Override public FileIO create(File file, OpenOption... openOption) throws IOException {
+            final FileIO delegate = delegateFactory.create(file, openOption);
 
             return new FileIODecorator(delegate) {
                 @Override public int write(ByteBuffer srcBuf) throws IOException {


[36/50] [abbrv] ignite git commit: ignite-5918 Adding and searching objects in index tree produces a lot of garbage

Posted by yz...@apache.org.
ignite-5918 Adding and searching objects in index tree produces a lot of garbage

(cherry picked from commit 0e0c2c7)

(cherry picked from commit 20739d8)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 71cd1e9188f59293c505dedd37c325687f3159f0
Parents: 212603e
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Wed Aug 9 18:46:53 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:28:26 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |   1 -
 .../query/h2/database/H2TreeIndex.java          |   6 +-
 .../query/h2/database/InlineIndexHelper.java    | 414 ++++++++++++++++++-
 .../h2/database/InlineIndexHelperTest.java      | 248 +++++++++--
 .../IgniteCacheWithIndexingTestSuite.java       |   3 +
 5 files changed, 642 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/71cd1e91/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 0cf2a82..973eeab 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -1261,7 +1261,6 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor, Object... args) throws TransactionException;
 
-
     /**
      * Invokes an {@link CacheEntryProcessor} against the set of {@link javax.cache.Cache.Entry}s
      * specified by the set of keys.

http://git-wip-us.apache.org/repos/asf/ignite/blob/71cd1e91/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 35bfdc1..de5dc75 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
@@ -145,7 +145,11 @@ public class H2TreeIndex extends GridH2IndexBase {
             if (!InlineIndexHelper.AVAILABLE_TYPES.contains(col.column.getType()))
                 break;
 
-            InlineIndexHelper idx = new InlineIndexHelper(col.column.getType(), col.column.getColumnId(), col.sortType);
+            InlineIndexHelper idx = new InlineIndexHelper(
+                col.column.getType(),
+                col.column.getColumnId(),
+                col.sortType,
+                table.getCompareMode());
 
             res.add(idx);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71cd1e91/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
index 19cf857..1789ac8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
@@ -23,8 +23,10 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.h2.result.SortOrder;
 import org.h2.table.IndexColumn;
+import org.h2.value.CompareMode;
 import org.h2.value.Value;
 import org.h2.value.ValueBoolean;
 import org.h2.value.ValueByte;
@@ -84,16 +86,28 @@ public class InlineIndexHelper {
     /** */
     private final short size;
 
+    /** */
+    private final boolean compareBinaryUnsigned;
+
+    /** */
+    private final boolean compareStringsOptimized;
+
     /**
      * @param type Index type (see {@link Value}).
      * @param colIdx Index column index.
      * @param sortType Column sort type (see {@link IndexColumn#sortType}).
      */
-    public InlineIndexHelper(int type, int colIdx, int sortType) {
+    public InlineIndexHelper(int type, int colIdx, int sortType, CompareMode compareMode) {
         this.type = type;
         this.colIdx = colIdx;
         this.sortType = sortType;
 
+        this.compareBinaryUnsigned = compareMode.isBinaryUnsigned();
+
+        // Optimized strings comparison can be used only if there are no custom collators.
+        // H2 internal comparison will be used otherwise (may be slower).
+        this.compareStringsOptimized = CompareMode.OFF.equals(compareMode.getName());
+
         switch (type) {
             case Value.BOOLEAN:
             case Value.BYTE:
@@ -330,13 +344,17 @@ public class InlineIndexHelper {
      * @return Compare result (-2 means we can't compare).
      */
     public int compare(long pageAddr, int off, int maxSize, Value v, Comparator<Value> comp) {
+        int c = tryCompareOptimized(pageAddr, off, maxSize, v);
+
+        if (c != Integer.MIN_VALUE)
+            return c;
+
         Value v1 = get(pageAddr, off, maxSize);
 
         if (v1 == null)
             return -2;
 
-        int c = comp.compare(v1, v);
-        c = c != 0 ? c > 0 ? 1 : -1 : 0;
+        c = Integer.signum(comp.compare(v1, v));
 
         if (size > 0)
             return fixSort(c, sortType());
@@ -350,6 +368,396 @@ public class InlineIndexHelper {
     /**
      * @param pageAddr Page address.
      * @param off Offset.
+     * @param maxSize Maximum size to read.
+     * @param v Value to compare.
+     * @return Compare result ({@code Integer.MIN_VALUE} means unsupported operation; {@code -2} - can't compare).
+     */
+    private int tryCompareOptimized(long pageAddr, int off, int maxSize, Value v) {
+        int type;
+
+        if ((size > 0 && size + 1 > maxSize)
+                || maxSize < 1
+                || (type = PageUtils.getByte(pageAddr, off)) == Value.UNKNOWN)
+            return -2;
+
+        if (type == Value.NULL)
+            return Integer.MIN_VALUE;
+
+        if (this.type != type)
+            throw new UnsupportedOperationException("Invalid fast index type: " + type);
+
+        type = Value.getHigherOrder(type, v.getType());
+
+        switch (type) {
+            case Value.BOOLEAN:
+            case Value.BYTE:
+            case Value.SHORT:
+            case Value.INT:
+            case Value.LONG:
+            case Value.FLOAT:
+            case Value.DOUBLE:
+                return compareAsPrimitive(pageAddr, off, v, type);
+
+            case Value.TIME:
+            case Value.DATE:
+            case Value.TIMESTAMP:
+                return compareAsDateTime(pageAddr, off, v, type);
+
+            case Value.STRING:
+            case Value.STRING_FIXED:
+            case Value.STRING_IGNORECASE:
+                if (compareStringsOptimized)
+                    return compareAsString(pageAddr, off, v, type == Value.STRING_IGNORECASE);
+
+                break;
+
+            case Value.BYTES:
+                return compareAsBytes(pageAddr, off, v);
+        }
+
+        return Integer.MIN_VALUE;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param v Value to compare.
+     * @param type Highest value type.
+     * @return Compare result ({@code -2} means we can't compare).
+     */
+    private int compareAsDateTime(long pageAddr, int off, Value v, int type) {
+        // only compatible types are supported now.
+        if(PageUtils.getByte(pageAddr, off) == type) {
+            switch (type) {
+                case Value.TIME:
+                    long nanos1 = PageUtils.getLong(pageAddr, off + 1);
+                    long nanos2 = ((ValueTime)v.convertTo(type)).getNanos();
+
+                    return fixSort(Long.signum(nanos1 - nanos2), sortType());
+
+                case Value.DATE:
+                    long date1 = PageUtils.getLong(pageAddr, off + 1);
+                    long date2 = ((ValueDate)v.convertTo(type)).getDateValue();
+
+                    return fixSort(Long.signum(date1 - date2), sortType());
+
+                case Value.TIMESTAMP:
+                    ValueTimestamp v0 = (ValueTimestamp) v.convertTo(type);
+
+                    date1 = PageUtils.getLong(pageAddr, off + 1);
+                    date2 = v0.getDateValue();
+
+                    int c = Long.signum(date1 - date2);
+
+                    if (c == 0) {
+                        nanos1 = PageUtils.getLong(pageAddr, off + 9);
+                        nanos2 = v0.getTimeNanos();
+
+                        c = Long.signum(nanos1 - nanos2);
+                    }
+
+                    return fixSort(c, sortType());
+            }
+        }
+
+        return Integer.MIN_VALUE;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param v Value to compare.
+     * @param type Highest value type.
+     * @return Compare result ({@code -2} means we can't compare).
+     */
+    private int compareAsPrimitive(long pageAddr, int off, Value v, int type) {
+        // only compatible types are supported now.
+        if(PageUtils.getByte(pageAddr, off) == type) {
+            switch (type) {
+                case Value.BOOLEAN:
+                    boolean bool1 = PageUtils.getByte(pageAddr, off + 1) != 0;
+                    boolean bool2 = v.getBoolean();
+
+                    return fixSort(Boolean.compare(bool1, bool2), sortType());
+
+                case Value.BYTE:
+                    byte byte1 = PageUtils.getByte(pageAddr, off + 1);
+                    byte byte2 = v.getByte();
+
+                    return fixSort(Integer.signum(byte1 - byte2), sortType());
+
+                case Value.SHORT:
+                    short short1 = PageUtils.getShort(pageAddr, off + 1);
+                    short short2 = v.getShort();
+
+                    return fixSort(Integer.signum(short1 - short2), sortType());
+
+                case Value.INT:
+                    int int1 = PageUtils.getInt(pageAddr, off + 1);
+                    int int2 = v.getInt();
+
+                    return fixSort(Integer.compare(int1, int2), sortType());
+
+                case Value.LONG:
+                    long long1 = PageUtils.getLong(pageAddr, off + 1);
+                    long long2 = v.getLong();
+
+                    return fixSort(Long.compare(long1, long2), sortType());
+
+                case Value.FLOAT:
+                    float float1 = Float.intBitsToFloat(PageUtils.getInt(pageAddr, off + 1));
+                    float float2 = v.getFloat();
+
+                    return fixSort(Float.compare(float1, float2), sortType());
+
+                case Value.DOUBLE:
+                    double double1 = Double.longBitsToDouble(PageUtils.getLong(pageAddr, off + 1));
+                    double double2 = v.getDouble();
+
+                    return fixSort(Double.compare(double1, double2), sortType());
+            }
+        }
+
+        return Integer.MIN_VALUE;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param v Value to compare.
+     * @return Compare result ({@code -2} means we can't compare).
+     */
+    private int compareAsBytes(long pageAddr, int off, Value v) {
+        byte[] bytes = v.getBytesNoCopy();
+
+        int len1;
+
+        long addr = pageAddr + off + 1; // Skip type.
+
+        if(size > 0)
+            // Fixed size value.
+            len1 = size;
+        else {
+            len1 = PageUtils.getShort(pageAddr, off + 1) & 0x7FFF;
+
+            addr += 2; // Skip size.
+        }
+
+        int len2 = bytes.length;
+
+        int len = Math.min(len1, len2);
+
+        if (compareBinaryUnsigned) {
+            for (int i = 0; i < len; i++) {
+                int b1 = GridUnsafe.getByte(addr + i) & 0xff;
+                int b2 = bytes[i] & 0xff;
+
+                if (b1 != b2)
+                    return fixSort(Integer.signum(b1 - b2), sortType());
+            }
+        }
+        else {
+            for (int i = 0; i < len; i++) {
+                byte b1 = GridUnsafe.getByte(addr + i);
+                byte b2 = bytes[i];
+
+                if (b1 != b2)
+                    return fixSort(Integer.signum(b1 - b2), sortType());
+            }
+        }
+
+        int res = Integer.signum(len1 - len2);
+
+        if(isValueFull(pageAddr, off))
+            return fixSort(res, sortType());
+
+        if (res >= 0)
+            // There are two cases:
+            // a) The values are equal but the stored value is truncated, so that it's bigger.
+            // b) Even truncated current value is longer, so that it's bigger.
+            return fixSort(1, sortType());
+
+        return -2;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param v Value to compare.
+     * @param ignoreCase {@code True} if a case-insensitive comparison should be used.
+     * @return Compare result ({@code -2} means we can't compare).
+     */
+    private int compareAsString(long pageAddr, int off, Value v, boolean ignoreCase) {
+        String s = v.getString();
+
+        int len1 = PageUtils.getShort(pageAddr, off + 1) & 0x7FFF;
+        int len2 = s.length();
+
+        int c, c2, c3, c4, cntr1 = 0, cntr2 = 0;
+        char v1, v2;
+
+        long addr = pageAddr + off + 3; // Skip length and type byte.
+
+        // Try reading ASCII.
+        while (cntr1 < len1 && cntr2 < len2) {
+            c = (int) GridUnsafe.getByte(addr) & 0xFF;
+
+            if (c > 127)
+                break;
+
+            cntr1++; addr++;
+
+            v1 = (char)c;
+            v2 = s.charAt(cntr2++);
+
+            if (ignoreCase) {
+                v1 = Character.toUpperCase(v1);
+                v2 = Character.toUpperCase(v2);
+            }
+
+            if (v1 != v2)
+                return fixSort(Integer.signum(v1 - v2), sortType());
+        }
+
+        // read other
+        while (cntr1 < len1 && cntr2 < len2) {
+            c = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+            switch (c >> 4) {
+                case 0:
+                case 1:
+                case 2:
+                case 3:
+                case 4:
+                case 5:
+                case 6:
+                case 7:
+                /* 0xxxxxxx*/
+                    cntr1++;
+
+                    v1 = (char)c;
+
+                    break;
+
+                case 12:
+                case 13:
+                /* 110x xxxx   10xx xxxx*/
+                    cntr1 += 2;
+
+                    if (cntr1 > len1)
+                        throw new IllegalStateException("Malformed input (partial character at the end).");
+
+                    c2 = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+                    if ((c2 & 0xC0) != 0x80)
+                        throw new IllegalStateException("Malformed input around byte: " + (cntr1 - 2));
+
+                    c = c & 0x1F;
+                    c = (c << 6) | (c2 & 0x3F);
+
+                    v1 = (char)c;
+
+                    break;
+
+                case 14:
+                /* 1110 xxxx  10xx xxxx  10xx xxxx */
+                    cntr1 += 3;
+
+                    if (cntr1 > len1)
+                        throw new IllegalStateException("Malformed input (partial character at the end).");
+
+                    c2 = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+                    c3 = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+                    if (((c2 & 0xC0) != 0x80) || ((c3 & 0xC0) != 0x80))
+                        throw new IllegalStateException("Malformed input around byte: " + (cntr1 - 3));
+
+                    c = c & 0x0F;
+                    c = (c << 6) | (c2 & 0x3F);
+                    c = (c << 6) | (c3 & 0x3F);
+
+                    v1 = (char)c;
+
+                    break;
+
+                case 15:
+                /* 11110xxx 10xxxxxx 10xxxxxx 10xxxxxx */
+                    cntr1 += 4;
+
+                    if (cntr1 > len1)
+                        throw new IllegalStateException("Malformed input (partial character at the end).");
+
+                    c2 = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+                    c3 = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+                    c4 = (int) GridUnsafe.getByte(addr++) & 0xFF;
+
+                    if (((c & 0xF8) != 0xf0) || ((c2 & 0xC0) != 0x80) || ((c3 & 0xC0) != 0x80) || ((c4 & 0xC0) != 0x80))
+                    throw new IllegalStateException("Malformed input around byte: " + (cntr1 - 4));
+
+                    c = c & 0x07;
+                    c = (c << 6) | (c2 & 0x3F);
+                    c = (c << 6) | (c3 & 0x3F);
+                    c = (c << 6) | (c4 & 0x3F);
+
+                    c = c - 0x010000; // Subtract 0x010000, c is now 0..fffff (20 bits)
+
+                    // height surrogate
+                    v1 = (char)(0xD800 + ((c >> 10) & 0x7FF));
+                    v2 = s.charAt(cntr2++);
+
+                    if (v1 != v2)
+                        return fixSort(Integer.signum(v1 - v2), sortType());
+
+                    if (cntr2 == len2)
+                        // The string is malformed (partial partial character at the end).
+                        // Finish comparison here.
+                        return fixSort(1, sortType());
+
+                    // Low surrogate.
+                    v1 = (char)(0xDC00 + (c & 0x3FF));
+                    v2 = s.charAt(cntr2++);
+
+                    if (v1 != v2)
+                        return fixSort(Integer.signum(v1 - v2), sortType());
+
+                    continue;
+
+                default:
+                /* 10xx xxxx */
+                    throw new IllegalStateException("Malformed input around byte: " + cntr1);
+            }
+
+            v2 = s.charAt(cntr2++);
+
+            if (ignoreCase) {
+                v1 = Character.toUpperCase(v1);
+                v2 = Character.toUpperCase(v2);
+            }
+
+            if (v1 != v2)
+                return fixSort(Integer.signum(v1 - v2), sortType());
+        }
+
+        int res = cntr1 == len1 && cntr2 == len2 ? 0 : cntr1 == len1 ? -1 : 1;
+
+        if (isValueFull(pageAddr, off))
+            return fixSort(res, sortType());
+
+        if (res >= 0)
+            // There are two cases:
+            // a) The values are equal but the stored value is truncated, so that it's bigger.
+            // b) Even truncated current value is longer, so that it's bigger.
+            return fixSort(1, sortType());
+
+        return -2;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param off Offset.
      * @param val Value.
      * @return NUmber of bytes saved.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/71cd1e91/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
index a2a3a72..fc06502 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -21,8 +21,9 @@ import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.UUID;
-import junit.framework.TestCase;
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.commons.io.Charsets;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
@@ -30,7 +31,7 @@ import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.persistence.MemoryMetricsImpl;
-import org.apache.ignite.logger.java.JavaLogger;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.h2.result.SortOrder;
 import org.h2.value.CompareMode;
 import org.h2.value.Value;
@@ -52,7 +53,7 @@ import org.h2.value.ValueUuid;
 /**
  * Simple tests for {@link InlineIndexHelper}.
  */
-public class InlineIndexHelperTest extends TestCase {
+public class InlineIndexHelperTest extends GridCommonAbstractTest {
     /** */
     private static final int CACHE_ID = 42;
 
@@ -79,17 +80,163 @@ public class InlineIndexHelperTest extends TestCase {
         assertEquals(4, bytes.length);
     }
 
+    /** */
+    public void testCompare1bytes() throws Exception {
+        int maxSize = 3 + 2; // 2 ascii chars + 3 bytes header.
+
+        assertEquals(0, putAndCompare("aa", "aa", maxSize));
+        assertEquals(-1, putAndCompare("aa", "bb", maxSize));
+        assertEquals(-1, putAndCompare("aaa", "bbb", maxSize));
+        assertEquals(1, putAndCompare("bbb", "aaa", maxSize));
+        assertEquals(1, putAndCompare("aaa", "aa", maxSize));
+        assertEquals(1, putAndCompare("aaa", "a", maxSize));
+        assertEquals(-2, putAndCompare("aaa", "aaa", maxSize));
+        assertEquals(-2, putAndCompare("aaa", "aab", maxSize));
+        assertEquals(-2, putAndCompare("aab", "aaa", maxSize));
+    }
+
+    /** */
+    public void testCompare2bytes() throws Exception {
+        int maxSize = 3 + 4; // 2 2-bytes chars + 3 bytes header.
+
+        assertEquals(0, putAndCompare("¡¡", "¡¡", maxSize));
+        assertEquals(-1, putAndCompare("¡¡", "¢¢", maxSize));
+        assertEquals(-1, putAndCompare("¡¡¡", "¢¢¢", maxSize));
+        assertEquals(1, putAndCompare("¢¢¢", "¡¡¡", maxSize));
+        assertEquals(1, putAndCompare("¡¡¡", "¡¡", maxSize));
+        assertEquals(1, putAndCompare("¡¡¡", "¡", maxSize));
+        assertEquals(-2, putAndCompare("¡¡¡", "¡¡¡", maxSize));
+        assertEquals(-2, putAndCompare("¡¡¡", "¡¡¢", maxSize));
+        assertEquals(-2, putAndCompare("¡¡¢", "¡¡¡", maxSize));
+    }
+
+    /** */
+    public void testCompare3bytes() throws Exception {
+        int maxSize = 3 + 6; // 2 3-bytes chars + 3 bytes header.
+
+        assertEquals(0, putAndCompare("ऄऄ", "ऄऄ", maxSize));
+        assertEquals(-1, putAndCompare("ऄऄ", "अअ", maxSize));
+        assertEquals(-1, putAndCompare("ऄऄऄ", "अअअ", maxSize));
+        assertEquals(1, putAndCompare("अअअ", "ऄऄऄ", maxSize));
+        assertEquals(1, putAndCompare("ऄऄऄ", "ऄऄ", maxSize));
+        assertEquals(1, putAndCompare("ऄऄऄ", "ऄ", maxSize));
+        assertEquals(-2, putAndCompare("ऄऄऄ", "ऄऄऄ", maxSize));
+        assertEquals(-2, putAndCompare("ऄऄऄ", "ऄऄअ", maxSize));
+        assertEquals(-2, putAndCompare("ऄऄअ", "ऄऄऄ", maxSize));
+    }
+
+    /** */
+    public void testCompare4bytes() throws Exception {
+        int maxSize = 3 + 8; // 2 4-bytes chars + 3 bytes header.
+
+        assertEquals(0, putAndCompare("\ud802\udd20\ud802\udd20", "\ud802\udd20\ud802\udd20", maxSize));
+        assertEquals(-1, putAndCompare("\ud802\udd20\ud802\udd20", "\ud802\udd21\ud802\udd21", maxSize));
+        assertEquals(-1, putAndCompare("\ud802\udd20\ud802\udd20\ud802\udd20", "\ud802\udd21\ud802\udd21\ud802\udd21", maxSize));
+        assertEquals(1, putAndCompare("\ud802\udd21\ud802\udd21\ud802\udd21", "\ud802\udd20\ud802\udd20\ud802\udd20", maxSize));
+        assertEquals(1, putAndCompare("\ud802\udd20\ud802\udd20\ud802\udd20", "\ud802\udd20\ud802\udd20", maxSize));
+        assertEquals(1, putAndCompare("\ud802\udd20\ud802\udd20\ud802\udd20", "\ud802\udd20", maxSize));
+        assertEquals(-2, putAndCompare("\ud802\udd20\ud802\udd20\ud802\udd20", "\ud802\udd20\ud802\udd20\ud802\udd20", maxSize));
+        assertEquals(-2, putAndCompare("\ud802\udd20\ud802\udd20\ud802\udd20", "\ud802\udd20\ud802\udd20\ud802\udd21", maxSize));
+        assertEquals(-2, putAndCompare("\ud802\udd20\ud802\udd20\ud802\udd21", "\ud802\udd20\ud802\udd20\ud802\udd20", maxSize));
+    }
+
+    /** */
+    public void testCompareMixed() throws Exception {
+        int maxSize = 3 + 8; // 2 up to 4-bytes chars + 3 bytes header.
+
+        assertEquals(0, putAndCompare("\ud802\udd20\u0904", "\ud802\udd20\u0904", maxSize));
+        assertEquals(-1, putAndCompare("\ud802\udd20\u0904", "\ud802\udd20\u0905", maxSize));
+        assertEquals(1, putAndCompare("\u0905\ud802\udd20", "\u0904\ud802\udd20", maxSize));
+        assertEquals(-2, putAndCompare("\ud802\udd20\ud802\udd20\u0905", "\ud802\udd20\ud802\udd20\u0904", maxSize));
+    }
+
+    /** */
+    public void testCompareMixed2() throws Exception {
+        int strCnt = 1000;
+        int symbCnt = 20;
+        int inlineSize = symbCnt * 4 + 3;
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        String[] strings = new String[strCnt];
+
+        for (int i = 0; i < strings.length; i++)
+            strings[i] = randomString(symbCnt);
+
+        Arrays.sort(strings);
+
+        for (int i = 0; i < 100; i++) {
+            int i1 = rnd.nextInt(strings.length);
+            int i2 = rnd.nextInt(strings.length);
+
+            assertEquals(Integer.compare(i1,i2), putAndCompare(strings[i1], strings[i2], inlineSize));
+        }
+    }
+
+    /**
+     * @param v1 Value 1.
+     * @param v2 Value 2.
+     * @param maxSize Max inline size.
+     * @return Compare result.
+     * @throws Exception If failed.
+     */
+    private int putAndCompare(String v1, String v2, int maxSize) throws Exception {
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setInitialSize(1024 * MB)
+            .setMaxSize(1024 * MB);
+
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
+                new UnsafeMemoryProvider(log),
+                null,
+                PAGE_SIZE,
+                plcCfg,
+                new MemoryMetricsImpl(plcCfg),
+                false);
+
+        pageMem.start();
+
+        long pageId = 0L;
+        long page = 0L;
+
+        try {
+            pageId = pageMem.allocatePage(CACHE_ID, 1, PageIdAllocator.FLAG_DATA);
+            page = pageMem.acquirePage(CACHE_ID, pageId);
+            long pageAddr = pageMem.readLock(CACHE_ID, pageId, page);
+
+            int off = 0;
+
+            InlineIndexHelper ih = new InlineIndexHelper(Value.STRING, 1, 0,
+                CompareMode.getInstance(null, 0));
+
+            ih.put(pageAddr, off, ValueString.get(v1), maxSize);
+
+            Comparator<Value> comp = new Comparator<Value>() {
+                @Override public int compare(Value o1, Value o2) {
+                    throw new AssertionError("Optimized algorithm should be used.");
+                }
+            };
+
+            return ih.compare(pageAddr, off, maxSize,  ValueString.get(v2), comp);
+        }
+        finally {
+            if (page != 0L)
+                pageMem.releasePage(CACHE_ID, pageId, page);
+
+            pageMem.stop();
+        }
+    }
+
     /** Limit is too small to cut */
     public void testStringCut() {
         // 6 bytes total: 3b, 3b.
-
         byte[] bytes = InlineIndexHelper.trimUTF8("\u20ac\u20ac".getBytes(Charsets.UTF_8), 2);
+
         assertNull(bytes);
     }
 
     /** Test on String values compare */
     public void testRelyOnCompare() {
-        InlineIndexHelper ha = new InlineIndexHelper(Value.STRING, 0, SortOrder.ASCENDING);
+        InlineIndexHelper ha = new InlineIndexHelper(Value.STRING, 0, SortOrder.ASCENDING,
+            CompareMode.getInstance(null, 0));
 
         // same size
         assertFalse(getRes(ha, "aabb", "aabb"));
@@ -110,7 +257,8 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** Test on Bytes values compare */
     public void testRelyOnCompareBytes() {
-        InlineIndexHelper ha = new InlineIndexHelper(Value.BYTES, 0, SortOrder.ASCENDING);
+        InlineIndexHelper ha = new InlineIndexHelper(Value.BYTES, 0, SortOrder.ASCENDING,
+            CompareMode.getInstance(null, 0));
 
         // same size
         assertFalse(getResBytes(ha, new byte[] {1, 2, 3, 4}, new byte[] {1, 2, 3, 4}));
@@ -131,12 +279,11 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** */
     public void testStringTruncate() throws Exception {
-        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
-
-        JavaLogger log = new JavaLogger();
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setInitialSize(1024 * MB)
+            .setMaxSize(1024 * MB);
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(log),
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log(),
+            new UnsafeMemoryProvider(log()),
             null,
             PAGE_SIZE,
             plcCfg,
@@ -155,7 +302,9 @@ public class InlineIndexHelperTest extends TestCase {
 
             int off = 0;
 
-            InlineIndexHelper ih = new InlineIndexHelper(Value.STRING, 1, 0);
+            InlineIndexHelper ih = new InlineIndexHelper(Value.STRING, 1, 0,
+                CompareMode.getInstance(null, 0));
+
             ih.put(pageAddr, off, ValueString.get("aaaaaaa"), 3 + 5);
 
             assertFalse(ih.isValueFull(pageAddr, off));
@@ -181,12 +330,11 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** */
     public void testBytes() throws Exception {
-        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setInitialSize(1024 * MB)
+            .setMaxSize(1024 * MB);
 
-        JavaLogger log = new JavaLogger();
-
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(log),
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log(),
+            new UnsafeMemoryProvider(log()),
             null,
             PAGE_SIZE,
             plcCfg,
@@ -205,7 +353,8 @@ public class InlineIndexHelperTest extends TestCase {
 
             int off = 0;
 
-            InlineIndexHelper ih = new InlineIndexHelper(Value.BYTES, 1, 0);
+            InlineIndexHelper ih = new InlineIndexHelper(Value.BYTES, 1, 0,
+                CompareMode.getInstance(null, 0));
 
             ih.put(pageAddr, off, ValueBytes.get(new byte[] {1, 2, 3, 4, 5}), 3 + 3);
 
@@ -296,12 +445,11 @@ public class InlineIndexHelperTest extends TestCase {
 
     /** */
     private void testPutGet(Value v1, Value v2, Value v3) throws Exception {
-        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setMaxSize(1024 * MB);
+        MemoryPolicyConfiguration plcCfg = new MemoryPolicyConfiguration().setInitialSize(1024 * MB)
+            .setMaxSize(1024 * MB);
 
-        JavaLogger log = new JavaLogger();
-
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log,
-            new UnsafeMemoryProvider(log),
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log(),
+            new UnsafeMemoryProvider(log()),
             null,
             PAGE_SIZE,
             plcCfg,
@@ -321,7 +469,8 @@ public class InlineIndexHelperTest extends TestCase {
             int off = 0;
             int max = 255;
 
-            InlineIndexHelper ih = new InlineIndexHelper(v1.getType(), 1, 0);
+            InlineIndexHelper ih = new InlineIndexHelper(v1.getType(), 1, 0,
+                CompareMode.getInstance(null, 0));
 
             off += ih.put(pageAddr, off, v1, max - off);
             off += ih.put(pageAddr, off, v2, max - off);
@@ -336,6 +485,7 @@ public class InlineIndexHelperTest extends TestCase {
         finally {
             if (page != 0L)
                 pageMem.releasePage(CACHE_ID, pageId, page);
+
             pageMem.stop();
         }
     }
@@ -346,6 +496,7 @@ public class InlineIndexHelperTest extends TestCase {
         Value v2 = s2 == null ? ValueNull.INSTANCE : ValueString.get(s2);
 
         int c = v1.compareTypeSafe(v2, CompareMode.getInstance(CompareMode.DEFAULT, 0));
+
         return ha.canRelyOnCompare(c, v1, v2);
     }
 
@@ -355,7 +506,54 @@ public class InlineIndexHelperTest extends TestCase {
         Value v2 = b2 == null ? ValueNull.INSTANCE : ValueBytes.get(b2);
 
         int c = v1.compareTypeSafe(v2, CompareMode.getInstance(CompareMode.DEFAULT, 0));
+
         return ha.canRelyOnCompare(c, v1, v2);
     }
 
-}
\ No newline at end of file
+    /**
+     * @param cnt String length.
+     * @return Random string.
+     */
+    private String randomString(int cnt) {
+        final char[] buffer = new char[cnt];
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        while (cnt-- != 0) {
+            char ch;
+
+            if (rnd.nextInt(100) > 3)
+                ch = (char) (rnd.nextInt(95) + 32); // regular symbols
+            else
+                ch = (char) (rnd.nextInt(65407) + 127); // others symbols
+
+            if(ch >= 56320 && ch <= 57343) {
+                if(cnt == 0)
+                    cnt++;
+                else {
+                    // low surrogate, insert high surrogate after putting it in
+                    buffer[cnt] = ch;
+                    cnt--;
+                    buffer[cnt] = (char) (55296 + rnd.nextInt(128));
+                }
+            }
+            else if(ch >= 55296 && ch <= 56191) {
+                if(cnt == 0)
+                    cnt++;
+                else {
+                    // high surrogate, insert low surrogate before putting it in
+                    buffer[cnt] = (char) (56320 + rnd.nextInt(128));
+                    cnt--;
+                    buffer[cnt] = ch;
+                }
+            }
+            else if(ch >= 56192 && ch <= 56319)
+                // private high surrogate, no effing clue, so skip it
+                cnt++;
+            else
+                buffer[cnt] = ch;
+        }
+
+        return new String(buffer);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71cd1e91/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 794ec4d..4047700 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
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.ttl.CacheTtlAtomicLocalSelfTe
 import org.apache.ignite.internal.processors.cache.ttl.CacheTtlAtomicPartitionedSelfTest;
 import org.apache.ignite.internal.processors.cache.ttl.CacheTtlTransactionalLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.ttl.CacheTtlTransactionalPartitionedSelfTest;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelperTest;
 
 /**
  * Cache tests using indexing.
@@ -49,6 +50,8 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite Cache With Indexing Test Suite");
 
+        suite.addTestSuite(InlineIndexHelperTest.class);
+
         suite.addTestSuite(GridIndexingWithNoopSwapSelfTest.class);
         suite.addTestSuite(GridCacheOffHeapSelfTest.class);
 


[49/50] [abbrv] ignite git commit: GG-12822 Moved notNullConstraint out of public API

Posted by yz...@apache.org.
GG-12822 Moved notNullConstraint out of public API


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: d9a2786d725d4649407b031d241c49da3c57fb79
Parents: d3a672e
Author: devozerov <pp...@gmail.com>
Authored: Wed Sep 27 17:23:48 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Sep 27 17:23:48 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/QueryEntity.java    |  25 +----
 .../processors/query/QueryEntityEx.java         | 103 +++++++++++++++++++
 .../internal/processors/query/QuerySchema.java  |  42 ++++++--
 .../internal/processors/query/QueryUtils.java   |  26 ++++-
 .../resources/META-INF/classnames.properties    |  10 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |  10 +-
 .../query/IgniteSqlNotNullConstraintTest.java   |   8 +-
 7 files changed, 182 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index fe54670..b824209 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -83,9 +83,6 @@ public class QueryEntity implements Serializable {
     /** Table name. */
     private String tableName;
 
-    /** Fields that must have non-null value. */
-    private Set<String> notNullFields;
-
     /**
      * Creates an empty query entity.
      */
@@ -112,8 +109,6 @@ public class QueryEntity implements Serializable {
         idxs = other.idxs != null ? new ArrayList<>(other.idxs) : null;
 
         tableName = other.tableName;
-
-        notNullFields = other.notNullFields != null ? new HashSet<>(other.notNullFields) : null;
     }
 
     /**
@@ -365,19 +360,7 @@ public class QueryEntity implements Serializable {
      * @return Set of names of fields that must have non-null values.
      */
     @Nullable public Set<String> getNotNullFields() {
-        return notNullFields;
-    }
-
-    /**
-     * Sets names of fields that must checked for null.
-     *
-     * @param notNullFields Set of names of fields that must have non-null values.
-     * @return {@code this} for chaining.
-     */
-    public QueryEntity setNotNullFields(@Nullable Set<String> notNullFields) {
-        this.notNullFields = notNullFields;
-
-        return this;
+        return null;
     }
 
     /**
@@ -620,6 +603,7 @@ public class QueryEntity implements Serializable {
 
         if (o == null || getClass() != o.getClass())
             return false;
+
         QueryEntity entity = (QueryEntity)o;
 
         return F.eq(keyType, entity.keyType) &&
@@ -630,14 +614,13 @@ public class QueryEntity implements Serializable {
             F.eq(keyFields, entity.keyFields) &&
             F.eq(aliases, entity.aliases) &&
             F.eqNotOrdered(idxs, entity.idxs) &&
-            F.eq(tableName, entity.tableName) &&
-            F.eq(notNullFields, entity.notNullFields);
+            F.eq(tableName, entity.tableName);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
         return Objects.hash(keyType, valType, keyFieldName, valueFieldName, fields, keyFields, aliases, idxs,
-            tableName, notNullFields);
+            tableName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.java
new file mode 100644
index 0000000..ec0d5fa
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEntityEx.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;
+
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Extended query entity with not-null fields support.
+ */
+public class QueryEntityEx extends QueryEntity {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Fields that must have non-null value. */
+    private Set<String> notNullFields;
+
+    /**
+     * Default constructor.
+     */
+    public QueryEntityEx() {
+        // No-op.
+    }
+
+    /**
+     * Copying constructor.
+     *
+     * @param other Instance to copy.
+     */
+    public QueryEntityEx(QueryEntity other) {
+        super(other);
+
+        if (other instanceof QueryEntityEx) {
+            QueryEntityEx other0 = (QueryEntityEx)other;
+
+            notNullFields = other0.notNullFields != null ? new HashSet<>(other0.notNullFields) : null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public Set<String> getNotNullFields() {
+        return notNullFields;
+    }
+
+    /**
+     * Sets names of fields that must checked for null.
+     *
+     * @param notNullFields Set of names of fields that must have non-null values.
+     * @return {@code this} for chaining.
+     */
+    public QueryEntity setNotNullFields(@Nullable Set<String> notNullFields) {
+        this.notNullFields = notNullFields;
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        QueryEntityEx entity = (QueryEntityEx)o;
+
+        return super.equals(entity) && F.eq(notNullFields, entity.notNullFields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = super.hashCode();
+
+        res = 31 * res + (notNullFields != null ? notNullFields.hashCode() : 0);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryEntityEx.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
index 40da780..62a9ecd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
@@ -23,6 +23,8 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
+
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
@@ -62,7 +64,7 @@ public class QuerySchema implements Serializable {
         assert entities != null;
 
         for (QueryEntity qryEntity : entities)
-            this.entities.add(new QueryEntity(qryEntity));
+            this.entities.add(QueryUtils.copy(qryEntity));
     }
 
     /**
@@ -75,7 +77,7 @@ public class QuerySchema implements Serializable {
             QuerySchema res = new QuerySchema();
 
             for (QueryEntity qryEntity : entities)
-                res.entities.add(new QueryEntity(qryEntity));
+                res.entities.add(QueryUtils.copy(qryEntity));
 
             return res;
         }
@@ -151,29 +153,51 @@ public class QuerySchema implements Serializable {
 
                 SchemaAlterTableAddColumnOperation op0 = (SchemaAlterTableAddColumnOperation)op;
 
-                QueryEntity target = null;
+                int targetIdx = -1;
+
+                for (int i = 0; i < entities.size(); i++) {
+                    QueryEntity entity = ((List<QueryEntity>)entities).get(i);
 
-                for (QueryEntity entity : entities()) {
                     if (F.eq(entity.getTableName(), op0.tableName())) {
-                        target = entity;
+                        targetIdx = i;
 
                         break;
                     }
                 }
 
-                if (target == null)
+                if (targetIdx == -1)
                     return;
 
+                boolean replaceTarget = false;
+
+                QueryEntity target = ((List<QueryEntity>)entities).get(targetIdx);
+
                 for (QueryField field : op0.columns()) {
                     target.getFields().put(field.name(), field.typeName());
 
                     if (!field.isNullable()) {
-                        if (target.getNotNullFields() == null)
-                            target.setNotNullFields(new HashSet<String>());
+                        if (!(target instanceof QueryEntityEx)) {
+                            target = new QueryEntityEx(target);
+
+                            replaceTarget = true;
+                        }
 
-                        target.getNotNullFields().add(field.name());
+                        QueryEntityEx target0 = (QueryEntityEx)target;
+
+                        Set<String> notNullFields = target0.getNotNullFields();
+
+                        if (notNullFields == null) {
+                            notNullFields = new HashSet<>();
+
+                            target0.setNotNullFields(notNullFields);
+                        }
+
+                        notNullFields.add(field.name());
                     }
                 }
+
+                if (replaceTarget)
+                    ((List<QueryEntity>)entities).set(targetIdx, target);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 76efb71..3e96db2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -222,7 +222,14 @@ public class QueryUtils {
         normalEntity.setKeyFields(entity.getKeyFields());
         normalEntity.setKeyFieldName(entity.getKeyFieldName());
         normalEntity.setValueFieldName(entity.getValueFieldName());
-        normalEntity.setNotNullFields(entity.getNotNullFields());
+
+        if (!F.isEmpty(entity.getNotNullFields())) {
+            QueryEntityEx normalEntity0 = new QueryEntityEx(normalEntity);
+
+            normalEntity0.setNotNullFields(entity.getNotNullFields());
+
+            normalEntity = normalEntity0;
+        }
 
         // Normalize table name.
         String normalTblName = entity.getTableName();
@@ -1187,6 +1194,23 @@ public class QueryUtils {
     }
 
     /**
+     * Copy query entity.
+     *
+     * @param entity Query entity.
+     * @return Copied entity.
+     */
+    public static QueryEntity copy(QueryEntity entity) {
+        QueryEntity res;
+
+        if (entity instanceof QueryEntityEx)
+            res = new QueryEntityEx(entity);
+        else
+            res = new QueryEntity(entity);
+
+        return res;
+    }
+
+    /**
      * Private constructor.
      */
     private QueryUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index fd60bd4..d99a6f3 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -427,9 +427,9 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$11
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$12
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$13
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$14
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$15
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$15$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$16
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$17$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$17
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$2
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$25$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$27
@@ -891,6 +891,7 @@ org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedM
 org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$CheckpointEntryType
 org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$RebalanceIteratorAdapter
 org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager$1
+org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory
 org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory
 org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory
 org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl$Segment
@@ -1349,6 +1350,7 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$8
 org.apache.ignite.internal.processors.query.GridQueryProcessor$9
 org.apache.ignite.internal.processors.query.GridQueryProcessor$SchemaOperation$1
 org.apache.ignite.internal.processors.query.IgniteSQLException
+org.apache.ignite.internal.processors.query.QueryEntityEx
 org.apache.ignite.internal.processors.query.QueryField
 org.apache.ignite.internal.processors.query.QueryIndexKey
 org.apache.ignite.internal.processors.query.QuerySchema
@@ -2096,7 +2098,3 @@ org.apache.ignite.transactions.TransactionRollbackException
 org.apache.ignite.transactions.TransactionState
 org.apache.ignite.transactions.TransactionTimeoutException
 org.apache.ignite.util.AttributeNodeFilter
-org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO
-org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory
-org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO
-org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index 03f4e1f..f00e28d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEntityEx;
 import org.apache.ignite.internal.processors.query.QueryField;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
@@ -362,7 +363,14 @@ public class DdlStatementsProcessor {
         res.setKeyType(keyTypeName);
 
         res.setKeyFields(createTbl.primaryKeyColumns());
-        res.setNotNullFields(notNullFields);
+
+        if (!F.isEmpty(notNullFields)) {
+            QueryEntityEx res0 = new QueryEntityEx(res);
+
+            res0.setNotNullFields(notNullFields);
+
+            res = res0;
+        }
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a2786d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
index dab474e..607dea8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
@@ -144,7 +144,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
         cfg.setAtomicityMode(atomicityMode);
         cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
-        QueryEntity qe = new QueryEntity(Integer.class, Person.class);
+        QueryEntityEx qe = new QueryEntityEx(new QueryEntity(Integer.class, Person.class));
 
         qe.setNotNullFields(Collections.singleton("name"));
 
@@ -188,7 +188,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
     /** */
     public void testQueryEntityGetSetNotNullFields() throws Exception {
-        QueryEntity qe = new QueryEntity();
+        QueryEntityEx qe = new QueryEntityEx();
 
         assertNull(qe.getNotNullFields());
 
@@ -205,9 +205,9 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
     /** */
     public void testQueryEntityEquals() throws Exception {
-        QueryEntity a = new QueryEntity();
+        QueryEntityEx a = new QueryEntityEx();
 
-        QueryEntity b = new QueryEntity();
+        QueryEntityEx b = new QueryEntityEx();
 
         assertEquals(a, b);
 


[31/50] [abbrv] ignite git commit: IGNITE-6334 Throttle writing threads during ongoing checkpoint - Fixes #2710.

Posted by yz...@apache.org.
IGNITE-6334 Throttle writing threads during ongoing checkpoint - Fixes #2710.

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/520c2e36
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/520c2e36
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/520c2e36

Branch: refs/heads/ignite-2.1.5-p1
Commit: 520c2e368baf4cb79912a14663ea3fd1c7da6487
Parents: 73e1578
Author: Ivan Rakov <iv...@gmail.com>
Authored: Fri Sep 22 12:40:22 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Sep 22 12:44:22 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   5 +
 .../PersistentStoreConfiguration.java           |  28 +-
 .../GridCacheDatabaseSharedManager.java         |  48 ++-
 .../persistence/pagemem/PageMemoryImpl.java     |  96 +++++-
 .../persistence/pagemem/PagesWriteThrottle.java | 104 ++++++
 .../pagemem/BPlusTreePageMemoryImplTest.java    |   4 +-
 .../BPlusTreeReuseListPageMemoryImplTest.java   |   3 +-
 .../MetadataStoragePageMemoryImplTest.java      |   4 +-
 .../pagemem/PageMemoryImplNoLoadTest.java       |   4 +-
 .../persistence/pagemem/PageMemoryImplTest.java |   4 +-
 .../pagemem/PagesWriteThrottleSandboxTest.java  | 264 +++++++++++++++
 .../pagemem/PagesWriteThrottleSmokeTest.java    | 322 +++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite.java   |   4 +
 13 files changed, 866 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/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 628b165..f627e24 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Properties;
 import javax.net.ssl.HostnameVerifier;
 import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
@@ -718,6 +719,10 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_WAL_LOG_TX_RECORDS = "IGNITE_WAL_LOG_TX_RECORDS";
 
+    /** If this property is set, {@link PersistentStoreConfiguration#writeThrottlingEnabled} will be overridden to true
+     * independent of initial value in configuration. */
+    public static final String IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED = "IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index abca5a5..c44e92d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -84,6 +84,9 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Default wal archive directory. */
     public static final String DFLT_WAL_ARCHIVE_PATH = "db/wal/archive";
 
+    /** Default write throttling enabled. */
+    public static final boolean DFLT_WRITE_THROTTLING_ENABLED = false;
+
     /** */
     private String persistenceStorePath;
 
@@ -162,6 +165,11 @@ public class PersistentStoreConfiguration implements Serializable {
     private long walAutoArchiveAfterInactivity = -1;
 
     /**
+     * If true, threads that generate dirty pages too fast during ongoing checkpoint will be throttled.
+     */
+    private boolean writeThrottlingEnabled = DFLT_WRITE_THROTTLING_ENABLED;
+
+    /**
      * Returns a path the root directory where the Persistent Store will persist data and indexes.
      */
     public String getPersistentStorePath() {
@@ -240,7 +248,7 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Sets a number of threads to use for the checkpointing purposes.
      *
-     * @param checkpointingThreads Number of checkpointing threads. One thread is used by default.
+     * @param checkpointingThreads Number of checkpointing threads. Four threads are used by default.
      * @return {@code this} for chaining.
      */
     public PersistentStoreConfiguration setCheckpointingThreads(int checkpointingThreads) {
@@ -402,6 +410,24 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
+     * Gets flag indicating whether write throttling is enabled.
+     */
+    public boolean isWriteThrottlingEnabled() {
+        return writeThrottlingEnabled;
+    }
+
+    /**
+     * Sets flag indicating whether write throttling is enabled.
+     *
+     * @param writeThrottlingEnabled Write throttling enabled flag.
+     */
+    public PersistentStoreConfiguration setWriteThrottlingEnabled(boolean writeThrottlingEnabled) {
+        this.writeThrottlingEnabled = writeThrottlingEnabled;
+
+        return this;
+    }
+
+    /**
      * Gets the length of the time interval for rate-based metrics. This interval defines a window over which
      * hits will be tracked. Default value is {@link #DFLT_RATE_TIME_INTERVAL_MILLIS}.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/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 277143c..1b5dae6 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
@@ -302,6 +302,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** */
     private ObjectName persistenceMetricsMbeanName;
 
+    /** Counter for written checkpoint pages. Not null only if checkpoint is running. */
+    private volatile AtomicInteger writtenPagesCntr = null;
+
+    /** Number of pages in current checkpoint. */
+    private volatile int currCheckpointPagesCnt;
+
     /**
      * @param ctx Kernal context.
      */
@@ -666,6 +672,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 "Checkpoint page buffer size is too big, setting to an adjusted cache size [size="
                     + U.readableSize(cacheSize, false) + ",  memPlc=" + plcCfg.getName() + ']');
 
+        boolean writeThrottlingEnabled = persistenceCfg.isWriteThrottlingEnabled();
+
+        if (IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED, false))
+            writeThrottlingEnabled = true;
+
         PageMemoryImpl pageMem = new PageMemoryImpl(
             memProvider,
             calculateFragmentSizes(
@@ -698,7 +709,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 }
             },
             this,
-            memMetrics
+            memMetrics,
+            writeThrottlingEnabled
         );
 
         memMetrics.pageMemory(pageMem);
@@ -940,7 +952,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     * Gets the checkpoint read lock. While this lock is held, checkpoint thread will not acquiSnapshotWorkerre memory state.
+     * Gets the checkpoint read lock. While this lock is held, checkpoint thread will not acquireSnapshotWorker memory state.
      */
     @SuppressWarnings("LockAcquiredButNotSafelyReleased")
     @Override public void checkpointReadLock() {
@@ -1902,6 +1914,20 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
+     * Counter for written checkpoint pages. Not null only if checkpoint is running.
+     */
+    public AtomicInteger writtenPagesCounter() {
+        return writtenPagesCntr;
+    }
+
+    /**
+     * @return Number of pages in current checkpoint. If checkpoint is not running, returns 0.
+     */
+    public int currentCheckpointPagesCount() {
+        return currCheckpointPagesCnt;
+    }
+
+    /**
      * @param cpTs Checkpoint timestamp.
      * @param cpId Checkpoint ID.
      * @param type Checkpoint type.
@@ -2034,6 +2060,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                 Checkpoint chp = markCheckpointBegin(tracker);
 
+                currCheckpointPagesCnt = chp.pagesSize;
+
+                writtenPagesCntr = new AtomicInteger();
+
                 boolean interrupted = true;
 
                 try {
@@ -2045,7 +2075,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             asyncRunner == null ? 1 : chp.cpPages.collectionsSize());
 
                         tracker.onPagesWriteStart();
-                        final AtomicInteger writtenPagesCtr = new AtomicInteger();
+
                         final int totalPagesToWriteCnt = chp.cpPages.size();
 
                         if (asyncRunner != null) {
@@ -2055,7 +2085,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                                     chp.cpPages.innerCollection(i),
                                     updStores,
                                     doneWriteFut,
-                                    writtenPagesCtr,
                                     totalPagesToWriteCnt
                                 );
 
@@ -2074,7 +2103,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                                 chp.cpPages,
                                 updStores,
                                 doneWriteFut,
-                                writtenPagesCtr,
                                 totalPagesToWriteCnt);
 
                             write.run();
@@ -2398,6 +2426,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                         chp.cpEntry.checkpointMark(),
                         null,
                         CheckpointEntryType.END);
+
+                writtenPagesCntr = null;
+
+                currCheckpointPagesCnt = 0;
             }
 
             checkpointHist.onCheckpointFinished(chp);
@@ -2494,9 +2526,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /** */
         private CountDownFuture doneFut;
 
-        /** Counter for all written pages. May be shared between several workers */
-        private AtomicInteger writtenPagesCntr;
-
         /** Total pages to write, counter may be greater than {@link #writePageIds} size*/
         private final int totalPagesToWrite;
 
@@ -2506,7 +2535,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
          * @param writePageIds Collection of page IDs to write.
          * @param updStores
          * @param doneFut
-         * @param writtenPagesCntr all written pages counter, may be shared between several write tasks
          * @param totalPagesToWrite total pages to be written under this checkpoint
          */
         private WriteCheckpointPages(
@@ -2514,13 +2542,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             final Collection<FullPageId> writePageIds,
             final GridConcurrentHashSet<PageStore> updStores,
             final CountDownFuture doneFut,
-            @NotNull final AtomicInteger writtenPagesCntr,
             final int totalPagesToWrite) {
             this.tracker = tracker;
             this.writePageIds = writePageIds;
             this.updStores = updStores;
             this.doneFut = doneFut;
-            this.writtenPagesCntr = writtenPagesCntr;
             this.totalPagesToWrite = totalPagesToWrite;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/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 dbb64f8..1da17b5 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
@@ -33,6 +33,7 @@ import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -56,6 +57,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.MemoryMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingPageIO;
@@ -179,6 +181,9 @@ public class PageMemoryImpl implements PageMemoryEx {
     /** State checker. */
     private final CheckpointLockStateChecker stateChecker;
 
+    /** Number of used pages in checkpoint buffer. */
+    private final AtomicInteger cpBufPagesCntr = new AtomicInteger(0);
+
     /** */
     private ExecutorService asyncRunner = new ThreadPoolExecutor(
         0,
@@ -217,6 +222,12 @@ public class PageMemoryImpl implements PageMemoryEx {
     /** Flush dirty page closure. When possible, will be called by evictPage(). */
     private final GridInClosure3X<Long, FullPageId, PageMemoryEx> changeTracker;
 
+    /** Pages write throttle. */
+    private PagesWriteThrottle writeThrottle;
+
+    /** Write throttle enabled flag. */
+    private boolean throttleEnabled;
+
     /**  */
     private boolean pageEvictWarned;
 
@@ -232,6 +243,7 @@ public class PageMemoryImpl implements PageMemoryEx {
      * @param pageSize Page size.
      * @param flushDirtyPage Callback invoked when a dirty page is evicted.
      * @param changeTracker Callback invoked to track changes in pages.
+     * @param throttleEnabled Write throttle enabled flag.
      */
     public PageMemoryImpl(
         DirectMemoryProvider directMemoryProvider,
@@ -241,7 +253,8 @@ public class PageMemoryImpl implements PageMemoryEx {
         GridInClosure3X<FullPageId, ByteBuffer, Integer> flushDirtyPage,
         GridInClosure3X<Long, FullPageId, PageMemoryEx> changeTracker,
         CheckpointLockStateChecker stateChecker,
-        MemoryMetricsImpl memMetrics
+        MemoryMetricsImpl memMetrics,
+        boolean throttleEnabled
     ) {
         assert sharedCtx != null;
 
@@ -253,6 +266,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         this.flushDirtyPage = flushDirtyPage;
         this.changeTracker = changeTracker;
         this.stateChecker = stateChecker;
+        this.throttleEnabled = throttleEnabled;
 
         storeMgr = sharedCtx.pageStore();
         walMgr = sharedCtx.wal();
@@ -290,7 +304,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         DirectMemoryRegion cpReg = regions.get(regs - 1);
 
-        checkpointPool = new PagePool(regs - 1, cpReg);
+        checkpointPool = new PagePool(regs - 1, cpReg, cpBufPagesCntr);
 
         long checkpointBuf = cpReg.size();
 
@@ -305,12 +319,14 @@ public class PageMemoryImpl implements PageMemoryEx {
 
             totalAllocated += reg.size();
 
-            segments[i] = new Segment(i, regions.get(i), checkpointPool.pages() / segments.length);
+            segments[i] = new Segment(i, regions.get(i), checkpointPool.pages() / segments.length, throttleEnabled);
 
             pages += segments[i].pages();
             totalTblSize += segments[i].tableSize();
         }
 
+        initWriteThrottle();
+
         if (log.isInfoEnabled())
             log.info("Started page memory [memoryAllocated=" + U.readableSize(totalAllocated, false) +
                 ", pages=" + pages +
@@ -319,6 +335,21 @@ public class PageMemoryImpl implements PageMemoryEx {
                 ']');
     }
 
+    /**
+     *
+     */
+    private void initWriteThrottle() {
+        if (!(sharedCtx.database() instanceof GridCacheDatabaseSharedManager)) {
+            log.error("Write throttle can't start. Unexpected class of database manager: " +
+                sharedCtx.database().getClass());
+
+            throttleEnabled = false;
+        }
+
+        if (throttleEnabled)
+            writeThrottle = new PagesWriteThrottle(this, (GridCacheDatabaseSharedManager)sharedCtx.database());
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("OverlyStrongTypeCast")
     @Override public void stop() throws IgniteException {
@@ -774,6 +805,18 @@ public class PageMemoryImpl implements PageMemoryEx {
         return true;
     }
 
+    /**
+     * @param dirtyRatioThreshold Throttle threshold.
+     */
+    boolean shouldThrottle(double dirtyRatioThreshold) {
+        for (Segment segment : segments) {
+            if (segment.shouldThrottle(dirtyRatioThreshold))
+                return true;
+        }
+
+        return false;
+    }
+
     /** {@inheritDoc} */
     @Override public GridMultiCollectionWrapper<FullPageId> beginCheckpoint() throws IgniteException {
         Collection[] collections = new Collection[segments.length];
@@ -799,6 +842,9 @@ public class PageMemoryImpl implements PageMemoryEx {
     @Override public void finishCheckpoint() {
         for (Segment seg : segments)
             seg.segCheckpointPages = null;
+
+        if (throttleEnabled)
+            writeThrottle.onFinishCheckpoint();
     }
 
     /** {@inheritDoc} */
@@ -1219,6 +1265,9 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         try {
             rwLock.writeUnlock(page + PAGE_LOCK_OFFSET, PageIdUtils.tag(pageId));
+
+            if (throttleEnabled && !restore && markDirty && !dirty)
+                writeThrottle.onMarkDirty(isInCheckpoint(fullId));
         }
         catch (AssertionError ex) {
             StringBuilder sb = new StringBuilder(sysPageSize * 2);
@@ -1310,6 +1359,20 @@ public class PageMemoryImpl implements PageMemoryEx {
     }
 
     /**
+     * Number of used pages in checkpoint buffer.
+     */
+    public int checkpointBufferPagesCount() {
+        return cpBufPagesCntr.get();
+    }
+
+    /**
+     * Number of used pages in checkpoint buffer.
+     */
+    public int checkpointBufferPagesSize() {
+        return checkpointPool.pages();
+    }
+
+    /**
      * This method must be called in synchronized context.
      *
      * @param absPtr Absolute pointer.
@@ -1385,6 +1448,9 @@ public class PageMemoryImpl implements PageMemoryEx {
         /** Direct memory region. */
         protected final DirectMemoryRegion region;
 
+        /** Pool pages counter. */
+        protected final AtomicInteger pagesCntr;
+
         /** */
         protected long lastAllocatedIdxPtr;
 
@@ -1397,10 +1463,12 @@ public class PageMemoryImpl implements PageMemoryEx {
         /**
          * @param idx Index.
          * @param region Region
+         * @param pagesCntr Pages counter.
          */
-        protected PagePool(int idx, DirectMemoryRegion region) {
+        protected PagePool(int idx, DirectMemoryRegion region, AtomicInteger pagesCntr) {
             this.idx = idx;
             this.region = region;
+            this.pagesCntr = pagesCntr;
 
             long base = (region.address() + 7) & ~0x7;
 
@@ -1427,6 +1495,9 @@ public class PageMemoryImpl implements PageMemoryEx {
          * @throws GridOffHeapOutOfMemoryException If failed to allocate new free page.
          */
         private long borrowOrAllocateFreePage(long pageId) throws GridOffHeapOutOfMemoryException {
+            if (pagesCntr != null)
+                pagesCntr.getAndIncrement();
+
             long relPtr = borrowFreePage();
 
             return relPtr != INVALID_REL_PTR ? relPtr : allocateFreePage(pageId);
@@ -1500,6 +1571,9 @@ public class PageMemoryImpl implements PageMemoryEx {
 
             assert !PageHeader.isAcquired(absPtr) : "Release pinned page: " + PageHeader.fullPageId(absPtr);
 
+            if (pagesCntr != null)
+                pagesCntr.getAndDecrement();
+
             while (true) {
                 long freePageRelPtrMasked = GridUnsafe.getLong(freePageListPtr);
 
@@ -1580,8 +1654,9 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         /**
          * @param region Memory region.
+         * @param throttlingEnabled Write throttling enabled flag.
          */
-        private Segment(int idx, DirectMemoryRegion region, int cpPoolPages) {
+        private Segment(int idx, DirectMemoryRegion region, int cpPoolPages, boolean throttlingEnabled) {
             long totalMemory = region.size();
 
             int pages = (int)(totalMemory / sysPageSize);
@@ -1596,9 +1671,9 @@ public class PageMemoryImpl implements PageMemoryEx {
 
             DirectMemoryRegion poolRegion = region.slice(memPerTbl + 8);
 
-            pool = new PagePool(idx, poolRegion);
+            pool = new PagePool(idx, poolRegion, null);
 
-            maxDirtyPages = Math.min(pool.pages() * 2 / 3, cpPoolPages);
+            maxDirtyPages = throttlingEnabled ? pool.pages() * 3 / 4 : Math.min(pool.pages() * 2 / 3, cpPoolPages);
         }
 
         /**
@@ -1609,6 +1684,13 @@ public class PageMemoryImpl implements PageMemoryEx {
         }
 
         /**
+         * @param dirtyRatioThreshold Throttle threshold.
+         */
+        private boolean shouldThrottle(double dirtyRatioThreshold) {
+            return ((double)dirtyPages.size()) / pages() > dirtyRatioThreshold;
+        }
+
+        /**
          * @return Max number of pages this segment can allocate.
          */
         private int pages() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java
new file mode 100644
index 0000000..d0c67c7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java
@@ -0,0 +1,104 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.ignite.internal.processors.cache.persistence.pagemem;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.LockSupport;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+
+/**
+ * Throttles threads that generate dirty pages during ongoing checkpoint.
+ * Designed to avoid zero dropdowns that can happen if checkpoint buffer is overflowed.
+ */
+public class PagesWriteThrottle {
+    /** Page memory. */
+    private final PageMemoryImpl pageMemory;
+
+    /** Database manager. */
+    private final GridCacheDatabaseSharedManager dbSharedMgr;
+
+    /** Starting throttle time. Limits write speed to 1000 MB/s. */
+    private static final long STARTING_THROTTLE_NANOS = 4000;
+
+    /** Backoff ratio. Each next park will be this times longer. */
+    private static final double BACKOFF_RATIO = 1.05;
+
+    /** Exponential backoff counter. */
+    private final AtomicInteger exponentialBackoffCntr = new AtomicInteger(0);
+    /**
+     * @param pageMemory Page memory.
+     * @param dbSharedMgr Database manager.
+     */
+    public PagesWriteThrottle(PageMemoryImpl pageMemory, GridCacheDatabaseSharedManager dbSharedMgr) {
+        this.pageMemory = pageMemory;
+        this.dbSharedMgr = dbSharedMgr;
+    }
+
+    /**
+     *
+     */
+    public void onMarkDirty(boolean isInCheckpoint) {
+        assert dbSharedMgr.checkpointLockIsHeldByThread();
+
+        AtomicInteger writtenPagesCntr = dbSharedMgr.writtenPagesCounter();
+
+        if (writtenPagesCntr == null)
+            return; // Don't throttle if checkpoint is not running.
+
+        boolean shouldThrottle = false;
+
+        if (isInCheckpoint) {
+            int checkpointBufLimit = pageMemory.checkpointBufferPagesSize() * 2 / 3;
+
+            shouldThrottle = pageMemory.checkpointBufferPagesCount() > checkpointBufLimit;
+        }
+
+        if (!shouldThrottle) {
+            int cpWrittenPages = writtenPagesCntr.get();
+
+            int cpTotalPages = dbSharedMgr.currentCheckpointPagesCount();
+
+            if (cpWrittenPages == cpTotalPages) {
+                // Checkpoint is already in fsync stage, increasing maximum ratio of dirty pages to 3/4
+                shouldThrottle = pageMemory.shouldThrottle(3.0 / 4);
+            } else {
+                double dirtyRatioThreshold = ((double)cpWrittenPages) / cpTotalPages;
+
+                // Starting with 0.05 to avoid throttle right after checkpoint start
+                // 7/12 is maximum ratio of dirty pages
+                dirtyRatioThreshold = (dirtyRatioThreshold * 0.95 + 0.05) * 7 / 12;
+
+                shouldThrottle = pageMemory.shouldThrottle(dirtyRatioThreshold);
+            }
+        }
+
+        if (shouldThrottle) {
+            int throttleLevel = exponentialBackoffCntr.getAndIncrement();
+
+            LockSupport.parkNanos((long)(STARTING_THROTTLE_NANOS * Math.pow(BACKOFF_RATIO, throttleLevel)));
+        }
+        else
+            exponentialBackoffCntr.set(0);
+    }
+
+    /**
+     *
+     */
+    public void onFinishCheckpoint() {
+        exponentialBackoffCntr.set(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
index 6f58782..56d09f8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
@@ -82,7 +82,9 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest {
                     return true;
                 }
             },
-            new MemoryMetricsImpl(new MemoryPolicyConfiguration()));
+            new MemoryMetricsImpl(new MemoryPolicyConfiguration()),
+            false
+        );
 
         mem.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
index b263d4f..39183b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
@@ -82,7 +82,8 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest
                     return true;
                 }
             },
-            new MemoryMetricsImpl(new MemoryPolicyConfiguration())
+            new MemoryMetricsImpl(new MemoryPolicyConfiguration()),
+            false
         );
 
         mem.start();

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/MetadataStoragePageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/MetadataStoragePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/MetadataStoragePageMemoryImplTest.java
index d9257bd..a427c63 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/MetadataStoragePageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/MetadataStoragePageMemoryImplTest.java
@@ -97,6 +97,8 @@ public class MetadataStoragePageMemoryImplTest extends MetadataStorageSelfTest{
                     return true;
                 }
             },
-            new MemoryMetricsImpl(new MemoryPolicyConfiguration()));
+            new MemoryMetricsImpl(new MemoryPolicyConfiguration()),
+            false
+        );
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
index 1fff1f0..467ede4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
@@ -88,7 +88,9 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest {
                     return true;
                 }
             },
-            new MemoryMetricsImpl(new MemoryPolicyConfiguration()));
+            new MemoryMetricsImpl(new MemoryPolicyConfiguration()),
+            false
+        );
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
index 0366eca..c5997fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
@@ -110,7 +110,9 @@ public class PageMemoryImplTest extends GridCommonAbstractTest {
                     return true;
                 }
             },
-            new MemoryMetricsImpl(new MemoryPolicyConfiguration()));
+            new MemoryMetricsImpl(new MemoryPolicyConfiguration()),
+            false
+        );
 
         mem.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
new file mode 100644
index 0000000..409ab84
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
@@ -0,0 +1,264 @@
+/*
+* 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.pagemem;
+
+import java.io.Serializable;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.MemoryMetrics;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+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.ratemetrics.HitRateMetrics;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+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;
+
+/**
+ * Test to visualize and debug {@link PagesWriteThrottle}.
+ * Prints puts/gets rate, number of dirty pages, pages written in current checkpoint and pages in checkpoint buffer.
+ * Not intended to be part of any test suite.
+ */
+public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache1";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoverySpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
+        discoverySpi.setIpFinder(ipFinder);
+
+        MemoryConfiguration dbCfg = new MemoryConfiguration();
+
+        dbCfg.setMemoryPolicies(new MemoryPolicyConfiguration()
+            .setMaxSize(4000L * 1024 * 1024)
+            .setName("dfltMemPlc")
+            .setMetricsEnabled(true));
+
+        dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+
+        cfg.setMemoryConfiguration(dbCfg);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setName(CACHE_NAME);
+        ccfg1.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg1.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg1.setAffinity(new RendezvousAffinityFunction(false, 64));
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        cfg.setPersistentStoreConfiguration(
+            new PersistentStoreConfiguration()
+                .setWalMode(WALMode.BACKGROUND)
+                .setCheckpointingFrequency(20_000)
+                .setCheckpointingPageBufferSize(1000L * 1000 * 1000)
+                .setWriteThrottlingEnabled(true));
+
+        cfg.setConsistentId(gridName);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 100 * 60 * 1000;
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testThrottle() throws Exception {
+        startGrids(1).active(true);
+
+        try {
+            final Ignite ig = ignite(0);
+
+            final int keyCnt = 4_000_000;
+
+            final AtomicBoolean run = new AtomicBoolean(true);
+
+            final HitRateMetrics getRate = new HitRateMetrics(5000, 5);
+
+            GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    while (run.get()) {
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        int key = rnd.nextInt(keyCnt * 2);
+
+                        ignite(0).cache(CACHE_NAME).get(key);
+
+                        getRate.onHit();
+                    }
+
+                    return null;
+                }
+            }, 2, "read-loader");
+
+            final HitRateMetrics putRate = new HitRateMetrics(1000, 5);
+
+            GridTestUtils.runAsync(new Runnable() {
+                @Override public void run() {
+                    while (run.get()) {
+                        long dirtyPages = 0;
+
+                        for (MemoryMetrics m : ig.memoryMetrics())
+                            if (m.getName().equals("dfltMemPlc"))
+                                dirtyPages = m.getDirtyPages();
+
+                        long cpBufPages = 0;
+
+                        long cpWrittenPages;
+
+                        AtomicInteger cntr = ((GridCacheDatabaseSharedManager)(((IgniteEx)ignite(0))
+                            .context().cache().context().database())).writtenPagesCounter();
+
+                        cpWrittenPages = cntr == null ? 0 : cntr.get();
+
+                        try {
+                            cpBufPages = ((PageMemoryImpl)((IgniteEx)ignite(0)).context().cache().context().database()
+                                .memoryPolicy("dfltMemPlc").pageMemory()).checkpointBufferPagesCount();
+                        }
+                        catch (IgniteCheckedException e) {
+                            e.printStackTrace();
+                        }
+
+                        System.out.println("@@@ putsPerSec=," + (putRate.getRate()) + ", getsPerSec=," + (getRate.getRate())  + ", dirtyPages=," + dirtyPages + ", cpWrittenPages=," + cpWrittenPages +", cpBufPages=," + cpBufPages);
+
+                        try {
+                            Thread.sleep(1000);
+                        }
+                        catch (InterruptedException e) {
+                            Thread.currentThread().interrupt();
+                        }
+                    }
+                }
+            }, "metrics-view");
+
+            try (IgniteDataStreamer<Object, Object> ds = ig.dataStreamer(CACHE_NAME)) {
+                ds.allowOverwrite(true);
+
+                for (int i = 0; i < keyCnt * 10; i++) {
+                    ds.addData(ThreadLocalRandom.current().nextInt(keyCnt), new TestValue(ThreadLocalRandom.current().nextInt(),
+                        ThreadLocalRandom.current().nextInt()));
+
+                    putRate.onHit();
+                }
+            }
+
+            run.set(false);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private final int v1;
+
+        /** */
+        private final int v2;
+
+        /** */
+        private byte[] payload = new byte[400 + ThreadLocalRandom.current().nextInt(20)];
+
+        /**
+         * @param v1 Value 1.
+         * @param v2 Value 2.
+         */
+        private TestValue(int v1, int v2) {
+            this.v1 = v1;
+            this.v2 = v2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue val = (TestValue)o;
+
+            return v1 == val.v1 && v2 == val.v2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = v1;
+
+            res = 31 * res + v2;
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
new file mode 100644
index 0000000..12a601d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
@@ -0,0 +1,322 @@
+/*
+* 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.pagemem;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.LockSupport;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
+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.ratemetrics.HitRateMetrics;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+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;
+
+/**
+ *
+ */
+public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Slow checkpoint enabled. */
+    private final AtomicBoolean slowCheckpointEnabled = new AtomicBoolean(true);
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache1";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoverySpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
+        discoverySpi.setIpFinder(ipFinder);
+
+        MemoryConfiguration dbCfg = new MemoryConfiguration();
+
+        dbCfg.setMemoryPolicies(new MemoryPolicyConfiguration()
+            .setMaxSize(400 * 1024 * 1024)
+            .setName("dfltMemPlc")
+            .setMetricsEnabled(true));
+
+        dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+
+        cfg.setMemoryConfiguration(dbCfg);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setName(CACHE_NAME);
+        ccfg1.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg1.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        ccfg1.setAffinity(new RendezvousAffinityFunction(false, 64));
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        cfg.setPersistentStoreConfiguration(
+            new PersistentStoreConfiguration()
+                .setWalMode(WALMode.BACKGROUND)
+                .setCheckpointingFrequency(20_000)
+                .setCheckpointingPageBufferSize(200 * 1000 * 1000)
+                .setWriteThrottlingEnabled(true)
+                .setCheckpointingThreads(1)
+                .setFileIOFactory(new SlowCheckpointFileIOFactory()));
+
+        cfg.setConsistentId(gridName);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        deleteWorkFiles();
+
+        slowCheckpointEnabled.set(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 6 * 60 * 1000;
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testThrottle() throws Exception {
+        startGrids(2).active(true);
+
+        try {
+            Ignite ig = ignite(0);
+
+            final int keyCnt = 2_000_000;
+
+            final AtomicBoolean run = new AtomicBoolean(true);
+
+            final AtomicBoolean zeroDropdown = new AtomicBoolean(false);
+
+            final HitRateMetrics putRate10secs = new HitRateMetrics(10_000, 20);
+
+            final HitRateMetrics putRate1sec = new HitRateMetrics(1_000, 20);
+
+            GridTestUtils.runAsync(new Runnable() {
+                @Override public void run() {
+                    try {
+                        Thread.sleep(5000);
+
+                        while (run.get()) {
+                            System.out.println(
+                                "Put rate over last 10 seconds: " + (putRate10secs.getRate() / 10) +
+                                    " puts/sec, over last 1 second: " + putRate1sec.getRate());
+
+                            if (putRate10secs.getRate() == 0) {
+                                zeroDropdown.set(true);
+
+                                run.set(false);
+                            }
+
+                            Thread.sleep(1000);
+                        }
+                    }
+                    catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                    finally {
+                        run.set(false);
+                    }
+                }
+            }, "rate-checker");
+
+            final IgniteCache<Integer, TestValue> cache = ig.getOrCreateCache(CACHE_NAME);
+
+            GridTestUtils.runAsync(new Runnable() {
+                @Override public void run() {
+                    long startTs = System.currentTimeMillis();
+
+                    for (int i = 0; i < keyCnt * 10 && System.currentTimeMillis() - startTs < 3 * 60 * 1000; i++) {
+                        if (!run.get())
+                            break;
+
+                        cache.put(ThreadLocalRandom.current().nextInt(keyCnt), new TestValue(ThreadLocalRandom.current().nextInt(),
+                            ThreadLocalRandom.current().nextInt()));
+
+                        putRate10secs.onHit();
+
+                        putRate1sec.onHit();
+                    }
+
+                    run.set(false);
+                }
+            }, "loader");
+
+            while (run.get())
+                LockSupport.parkNanos(10_000);
+
+            if (zeroDropdown.get()) {
+                slowCheckpointEnabled.set(false);
+
+                IgniteInternalFuture cpFut1 = ((IgniteEx)ignite(0)).context().cache().context().database()
+                    .wakeupForCheckpoint("test");
+
+                IgniteInternalFuture cpFut2 = ((IgniteEx)ignite(1)).context().cache().context().database()
+                    .wakeupForCheckpoint("test");
+
+                cpFut1.get();
+
+                cpFut2.get();
+
+                fail("Put rate degraded to zero for at least 10 seconds");
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private final int v1;
+
+        /** */
+        private final int v2;
+
+        /** */
+        private byte[] payload = new byte[400 + ThreadLocalRandom.current().nextInt(20)];
+
+        /**
+         * @param v1 Value 1.
+         * @param v2 Value 2.
+         */
+        private TestValue(int v1, int v2) {
+            this.v1 = v1;
+            this.v2 = v2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue val = (TestValue)o;
+
+            return v1 == val.v1 && v2 == val.v2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = v1;
+
+            res = 31 * res + v2;
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false));
+    }
+
+    /**
+     * Create File I/O that emulates poor checkpoint write speed.
+     */
+    private class SlowCheckpointFileIOFactory implements FileIOFactory {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** Delegate factory. */
+        private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory();
+
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file) throws IOException {
+            return create(file, "rw");
+        }
+
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file, String mode) throws IOException {
+            final FileIO delegate = delegateFactory.create(file, mode);
+
+            return new FileIODecorator(delegate) {
+                @Override public int write(ByteBuffer srcBuf) throws IOException {
+                    if (slowCheckpointEnabled.get() && Thread.currentThread().getName().contains("checkpoint"))
+                        LockSupport.parkNanos(5_000_000);
+
+                    return delegate.write(srcBuf);
+                }
+
+                @Override public int write(ByteBuffer srcBuf, long position) throws IOException {
+                    if (slowCheckpointEnabled.get() && Thread.currentThread().getName().contains("checkpoint"))
+                        LockSupport.parkNanos(5_000_000);
+
+                    return delegate.write(srcBuf, position);
+                }
+
+                @Override public void write(byte[] buf, int off, int len) throws IOException {
+                    if (slowCheckpointEnabled.get() && Thread.currentThread().getName().contains("checkpoint"))
+                        LockSupport.parkNanos(5_000_000);
+
+                    delegate.write(buf, off, len);
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/520c2e36/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
index b2a1f65..ef7682f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.BPlusTree
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.MetadataStoragePageMemoryImplTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImplNoLoadTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImplTest;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PagesWriteThrottleSmokeTest;
 import org.apache.ignite.internal.processors.database.IgniteDbClientNearCachePutGetTest;
 import org.apache.ignite.internal.processors.database.IgniteDbDynamicCacheSelfTest;
 import org.apache.ignite.internal.processors.database.IgniteDbMultiNodePutGetTest;
@@ -80,6 +81,9 @@ public class IgnitePdsTestSuite extends TestSuite {
 
         suite.addTestSuite(DefaultPageSizeBackwardsCompatibilityTest.class);
 
+        // Write throttling
+        suite.addTestSuite(PagesWriteThrottleSmokeTest.class);
+
         return suite;
     }
 }


[16/50] [abbrv] ignite git commit: IGNITE-6457: JDBC thin driver: now schema in URL string is case-insensitive by default. This closes #2712.

Posted by yz...@apache.org.
IGNITE-6457: JDBC thin driver: now schema in URL string is case-insensitive by default. This closes #2712.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 931320f6ded6f927931c6dca2b9ed6795632365c
Parents: 3847b47
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Sep 21 14:40:13 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Sep 21 14:41:30 2017 +0300

----------------------------------------------------------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   2 +
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |   6 +-
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |   6 +-
 .../jdbc/thin/JdbcThinSchemaCaseTest.java       | 129 +++++++++++++++++++
 .../org/apache/ignite/IgniteJdbcThinDriver.java |   1 -
 .../internal/jdbc/thin/JdbcThinConnection.java  |  23 +++-
 6 files changed, 159 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/931320f6/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 3d3ef27..0bd86b3 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -53,6 +53,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinMissingLongArrayResultsTest;
 import org.apache.ignite.jdbc.thin.JdbcThinNoDefaultSchemaTest;
 import org.apache.ignite.jdbc.thin.JdbcThinPreparedStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinResultSetSelfTest;
+import org.apache.ignite.jdbc.thin.JdbcThinSchemaCaseTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSelfTest;
 
@@ -122,6 +123,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(JdbcThinStatementSelfTest.class));
         suite.addTest(new TestSuite(JdbcThinComplexQuerySelfTest.class));
         suite.addTest(new TestSuite(JdbcThinNoDefaultSchemaTest.class));
+        suite.addTest(new TestSuite(JdbcThinSchemaCaseTest.class));
         suite.addTest(new TestSuite(JdbcThinEmptyCacheSelfTest.class));
         suite.addTest(new TestSuite(JdbcThinMetadataSelfTest.class));
         suite.addTest(new TestSuite(JdbcThinErrorsSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/931320f6/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 95068c5..17ce686 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
@@ -324,15 +324,15 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
             "Invalid URL format (only schema name is allowed in URL path parameter 'host:port[/schemaName]')" );
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/public")) {
-            assertEquals("Invalid schema", "public", conn.getSchema());
+            assertEquals("Invalid schema", "PUBLIC", conn.getSchema());
         }
 
-        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/" + DEFAULT_CACHE_NAME)) {
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/\"" + DEFAULT_CACHE_NAME + '"')) {
             assertEquals("Invalid schema", DEFAULT_CACHE_NAME, conn.getSchema());
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/_not_exist_schema_")) {
-            assertEquals("Invalid schema", "_not_exist_schema_", conn.getSchema());
+            assertEquals("Invalid schema", "_NOT_EXIST_SCHEMA_", conn.getSchema());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/931320f6/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
index 65ebf9a..cab28f4 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
@@ -161,7 +161,7 @@ public class JdbcThinNoDefaultSchemaTest extends JdbcThinAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testSchemaInUrl() throws Exception {
-        try(Connection conn = DriverManager.getConnection(URL + "/cache1")) {
+        try(Connection conn = DriverManager.getConnection(URL + "/\"cache1\"")) {
             Statement stmt = conn.createStatement();
 
             stmt.execute("select t._key, t._val from Integer t");
@@ -172,7 +172,7 @@ public class JdbcThinNoDefaultSchemaTest extends JdbcThinAbstractSelfTest {
                 assertEquals(rs.getInt(2), rs.getInt(1) * 2);
         }
 
-        try(Connection conn = DriverManager.getConnection(URL + "/cache2")) {
+        try(Connection conn = DriverManager.getConnection(URL + "/\"cache2\"")) {
             Statement stmt = conn.createStatement();
 
             stmt.execute("select t._key, t._val from Integer t");
@@ -188,7 +188,7 @@ public class JdbcThinNoDefaultSchemaTest extends JdbcThinAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testSchemaInUrlAndInQuery() throws Exception {
-        try(Connection conn = DriverManager.getConnection(URL + "/cache2")) {
+        try(Connection conn = DriverManager.getConnection(URL + "/\"cache2\"")) {
             Statement stmt = conn.createStatement();
 
             stmt.execute("select t._key, t._val, v._val " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/931320f6/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSchemaCaseTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSchemaCaseTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSchemaCaseTest.java
new file mode 100644
index 0000000..11416e5
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSchemaCaseTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.concurrent.Callable;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public class JdbcThinSchemaCaseTest extends JdbcThinAbstractSelfTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** URL. */
+    private static final String URL = "jdbc:ignite:thin://127.0.0.1";
+
+    /** Grid count. */
+    private static final int GRID_CNT = 2;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(
+            cacheConfiguration("test0", "test0"),
+            cacheConfiguration("test1", "tEst1"),
+            cacheConfiguration("test2", "\"TestCase\""));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @param name Cache name.
+     * @param schema Schema name.
+     * @return Cache configuration.
+     * @throws Exception In case of error.
+     */
+    @SuppressWarnings("unchecked")
+    private CacheConfiguration cacheConfiguration(@NotNull String name, @NotNull String schema) throws Exception {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setIndexedTypes(Integer.class, Integer.class);
+
+        cfg.setName(name);
+
+        cfg.setSqlSchema(schema);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"EmptyTryBlock", "unused"})
+    public void testSchemaName() throws Exception {
+        checkSchemaConnection("test0");
+        checkSchemaConnection("test1");
+        checkSchemaConnection("\"TestCase\"");
+        checkSchemaConnection("\"TEST0\"");
+        checkSchemaConnection("\"TEST1\"");
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                checkSchemaConnection("TestCase");
+
+                return null;
+            }
+        }, SQLException.class, null);
+    }
+
+    /**
+     * @param schema Schema name.
+     * @throws SQLException If failed.
+     */
+    void checkSchemaConnection(String schema) throws SQLException {
+        try (Connection conn = DriverManager.getConnection(URL + '/' + schema)) {
+            Statement stmt = conn.createStatement();
+
+            assertNotNull(stmt);
+            assertFalse(stmt.isClosed());
+
+            stmt.execute("select t._key, t._val from Integer t");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/931320f6/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
index 423223c..8085ed4 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
@@ -26,7 +26,6 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
-import java.util.logging.Level;
 import java.util.logging.Logger;
 import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.internal.IgniteVersionUtils;

http://git-wip-us.apache.org/repos/asf/ignite/blob/931320f6/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index 511eba0..d804750 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteProductVersion;
 
@@ -119,7 +120,7 @@ public class JdbcThinConnection implements Connection {
         autoCommit = true;
         txIsolation = Connection.TRANSACTION_NONE;
 
-        this.schema = schema == null ? "PUBLIC" : schema;
+        this.schema = normalizeSchema(schema);
 
         String host = extractHost(props);
         int port = extractPort(props);
@@ -799,4 +800,24 @@ public class JdbcThinConnection implements Connection {
     public String url() {
         return url;
     }
+
+    /**
+     * Normalize schema name. If it is quoted - unquote and leave as is, otherwise - convert to upper case.
+     *
+     * @param schemaName Schema name.
+     * @return Normalized schema name.
+     */
+    private static String normalizeSchema(String schemaName) {
+        if (F.isEmpty(schemaName))
+            return QueryUtils.DFLT_SCHEMA;
+
+        String res;
+
+        if (schemaName.startsWith("\"") && schemaName.endsWith("\""))
+            res = schemaName.substring(1, schemaName.length() - 1);
+        else
+            res = schemaName.toUpperCase();
+
+        return res;
+    }
 }
\ No newline at end of file


[28/50] [abbrv] ignite git commit: Versions updated.

Posted by yz...@apache.org.
Versions updated.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: cd4e25ecb4912beb85a571b33e0f1096a24bbb7c
Parents: ccc8c9b
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Sep 22 10:10:52 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Sep 22 10:10:52 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/services/Version.service.js | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cd4e25ec/modules/web-console/frontend/app/services/Version.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Version.service.js b/modules/web-console/frontend/app/services/Version.service.js
index 8ae23d3..beb27fa 100644
--- a/modules/web-console/frontend/app/services/Version.service.js
+++ b/modules/web-console/frontend/app/services/Version.service.js
@@ -73,12 +73,12 @@ const compare = (a, b) => {
 
 export default class IgniteVersion {
     constructor() {
-        this.webConsole = '2.1.0';
+        this.webConsole = '2.2.0';
 
         this.supportedVersions = [
             {
-                label: 'Ignite 2.1',
-                ignite: '2.1.0'
+                label: 'Ignite 2.x',
+                ignite: '2.2.0'
             },
             {
                 label: 'Ignite 2.0',


[15/50] [abbrv] ignite git commit: IGNITE-6461 Web Console: Sanitize user on save. (cherry picked from commit 1b6873c)

Posted by yz...@apache.org.
IGNITE-6461 Web Console: Sanitize user on save.
(cherry picked from commit 1b6873c)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 3847b47752db7d83c85a76e09964fd154e2f0cc4
Parents: 58ba044
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Sep 21 14:36:08 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Sep 21 14:38:02 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/routes/admin.js             |  5 +++--
 modules/web-console/backend/services/users.js           |  5 +++++
 .../list-of-registered-users.controller.js              |  1 -
 .../web-console/frontend/app/core/admin/Admin.data.js   | 12 ++++++++----
 4 files changed, 16 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3847b477/modules/web-console/backend/routes/admin.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/admin.js b/modules/web-console/backend/routes/admin.js
index c00b17a..5ee41c8 100644
--- a/modules/web-console/backend/routes/admin.js
+++ b/modules/web-console/backend/routes/admin.js
@@ -33,6 +33,7 @@ module.exports = {
  * @param {MailsService} mailsService
  * @param {SessionsService} sessionsService
  * @param {UsersService} usersService
+ * @param {NotificationsService} notificationsService
  * @returns {Promise}
  */
 module.exports.factory = function(_, express, settings, mongo, spacesService, mailsService, sessionsService, usersService, notificationsService) {
@@ -55,8 +56,8 @@ module.exports.factory = function(_, express, settings, mongo, spacesService, ma
                 .catch(res.api.error);
         });
 
-        // Save user.
-        router.post('/save', (req, res) => {
+        // Grant or revoke admin access to user.
+        router.post('/toggle', (req, res) => {
             const params = req.body;
 
             mongo.Account.findByIdAndUpdate(params.userId, {admin: params.adminFlag}).exec()

http://git-wip-us.apache.org/repos/asf/ignite/blob/3847b477/modules/web-console/backend/services/users.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/users.js b/modules/web-console/backend/services/users.js
index 0710713..991928a 100644
--- a/modules/web-console/backend/services/users.js
+++ b/modules/web-console/backend/services/users.js
@@ -51,6 +51,7 @@ module.exports.factory = (_, errors, settings, mongo, spacesService, mailsServic
     class UsersService {
         /**
          * Save profile information.
+         *
          * @param {String} host - The host
          * @param {Object} user - The user
          * @returns {Promise.<mongo.ObjectId>} that resolves account id of merge operation.
@@ -93,10 +94,13 @@ module.exports.factory = (_, errors, settings, mongo, spacesService, mailsServic
 
         /**
          * Save user.
+         *
          * @param {Object} changed - The user
          * @returns {Promise.<mongo.ObjectId>} that resolves account id of merge operation.
          */
         static save(changed) {
+            delete changed.admin;
+
             return mongo.Account.findById(changed._id).exec()
                 .then((user) => {
                     if (!changed.password)
@@ -207,6 +211,7 @@ module.exports.factory = (_, errors, settings, mongo, spacesService, mailsServic
 
         /**
          * Remove account.
+         *
          * @param {String} host.
          * @param {mongo.ObjectId|String} userId - The account id for remove.
          * @returns {Promise.<{rowsAffected}>} - The number of affected rows.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3847b477/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
index 2e36c5c..f82ccbe 100644
--- a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
+++ b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.controller.js
@@ -91,7 +91,6 @@ export default class IgniteListOfRegisteredUsersCtrl {
             user.adminChanging = true;
 
             AdminData.toggleAdmin(user)
-                .then(() => user.admin = !user.admin)
                 .finally(() => user.adminChanging = false);
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3847b477/modules/web-console/frontend/app/core/admin/Admin.data.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/core/admin/Admin.data.js b/modules/web-console/frontend/app/core/admin/Admin.data.js
index 5ea8f00..5c4fe10 100644
--- a/modules/web-console/frontend/app/core/admin/Admin.data.js
+++ b/modules/web-console/frontend/app/core/admin/Admin.data.js
@@ -47,15 +47,19 @@ export default class IgniteAdminData {
     }
 
     toggleAdmin(user) {
-        return this.$http.post('/api/v1/admin/save', {
+        const adminFlag = !user.admin;
+
+        return this.$http.post('/api/v1/admin/toggle', {
             userId: user._id,
-            adminFlag: !user.admin
+            adminFlag
         })
         .then(() => {
-            this.Messages.showInfo(`Admin right was successfully toggled for user: "${user.userName}"`);
+            user.admin = adminFlag;
+
+            this.Messages.showInfo(`Admin rights was successfully ${adminFlag ? 'granted' : 'revoked'} for user: "${user.userName}"`);
         })
         .catch((res) => {
-            this.Messages.showError('Failed to toggle admin right for user: ', res);
+            this.Messages.showError(`Failed to ${adminFlag ? 'grant' : 'revok'} admin rights for user: "${user.userName}"`, res);
         });
     }
 


[06/50] [abbrv] ignite git commit: IGNITE-6099: ODBC: Implemented SQLGetInfo for all info types. This closes #2689.

Posted by yz...@apache.org.
IGNITE-6099: ODBC: Implemented SQLGetInfo for all info types. This closes #2689.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 718e365f47bb5c91bc910854a7626b1933c83e59
Parents: be60c08
Author: Igor Sapego <ig...@gmail.com>
Authored: Tue Sep 19 16:23:54 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Sep 19 16:25:51 2017 +0300

----------------------------------------------------------------------
 .../cpp/odbc-test/src/connection_info_test.cpp  |    3 +-
 .../ignite/odbc/config/connection_info.h        |    8 +-
 .../cpp/odbc/include/ignite/odbc/connection.h   |    3 +
 .../cpp/odbc/src/config/connection_info.cpp     | 2235 ++++++++++++++++--
 modules/platforms/cpp/odbc/src/connection.cpp   |    6 +-
 5 files changed, 1997 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/718e365f/modules/platforms/cpp/odbc-test/src/connection_info_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/connection_info_test.cpp b/modules/platforms/cpp/odbc-test/src/connection_info_test.cpp
index c2fe4f8..2119c09 100644
--- a/modules/platforms/cpp/odbc-test/src/connection_info_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/connection_info_test.cpp
@@ -36,7 +36,8 @@ BOOST_AUTO_TEST_CASE(TestConnectionInfoSupportedInfo)
     char buffer[4096];
     short reslen = 0;
 
-    ConnectionInfo info;
+    Configuration cfg;
+    ConnectionInfo info(cfg);
 
     SqlResult::Type result;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/718e365f/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
index c17d5ec..fb6351a 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
@@ -24,6 +24,7 @@
 
 #include <ignite/common/common.h>
 #include <ignite/odbc/common_types.h>
+#include <ignite/odbc/config/configuration.h>
 
 namespace ignite
 {
@@ -50,8 +51,10 @@ namespace ignite
 
                 /**
                  * Constructor.
+                 *
+                 * @param config Configuration.
                  */
-                ConnectionInfo();
+                ConnectionInfo(const Configuration& config);
 
                 /**
                  * Destructor.
@@ -88,6 +91,9 @@ namespace ignite
 
                 /** Short parameters. */
                 UshortInfoMap shortParams;
+
+                /** Configuration. */
+                const Configuration& config;
             };
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/718e365f/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
index fa80a01..34fed5f 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
@@ -316,6 +316,9 @@ namespace ignite
 
             /** Configuration. */
             config::Configuration config;
+
+            /** Connection info. */
+            config::ConnectionInfo info;
         };
     }
 }


[37/50] [abbrv] ignite git commit: IGNITE-6051 Improve future listeners model in DataStreamerImpl

Posted by yz...@apache.org.
IGNITE-6051 Improve future listeners model in DataStreamerImpl

(cherry picked from commit 18ca0b2)

(cherry picked from commit cec55c3)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 1faa8dbed4009b691ee0fd5920aead8ec3e6719c
Parents: 71cd1e9
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Aug 21 17:28:25 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:28:40 2017 +0300

----------------------------------------------------------------------
 .../datastreamer/DataStreamerImpl.java          |  63 ++++++--
 .../util/future/GridCompoundFuture.java         |  40 ++++--
 .../datastreamer/DataStreamerImplSelfTest.java  | 143 ++++++++++++++++++-
 3 files changed, 221 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1faa8dbe/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 1869dcf..6ed552a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -941,10 +941,16 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                         }
                     };
 
+                    GridCompoundFuture opFut = new SilentCompoundFuture();
+
+                    opFut.listen(lsnr);
+
                     final List<GridFutureAdapter<?>> futs;
 
                     try {
-                        futs = buf.update(entriesForNode, topVer, lsnr, remap);
+                        futs = buf.update(entriesForNode, topVer, opFut, remap);
+
+                        opFut.markInitialized();
                     }
                     catch (IgniteInterruptedCheckedException e1) {
                         resFut.onDone(e1);
@@ -1382,7 +1388,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         /**
          * @param newEntries Infos.
          * @param topVer Topology version.
-         * @param lsnr Listener for the operation future.
+         * @param opFut Completion future for the operation.
          * @param remap Remapping flag.
          * @return Future for operation.
          * @throws IgniteInterruptedCheckedException If failed.
@@ -1390,10 +1396,11 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         @Nullable List<GridFutureAdapter<?>> update(
             Iterable<DataStreamerEntry> newEntries,
             AffinityTopologyVersion topVer,
-            IgniteInClosure<IgniteInternalFuture<?>> lsnr,
+            GridCompoundFuture opFut,
             boolean remap
         ) throws IgniteInterruptedCheckedException {
             List<GridFutureAdapter<?>> res = null;
+            GridFutureAdapter[] futs = new GridFutureAdapter[stripes.length];
 
             for (DataStreamerEntry entry : newEntries) {
                 List<DataStreamerEntry> entries0 = null;
@@ -1408,9 +1415,16 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 synchronized (b) {
                     curFut0 = b.curFut;
 
-                    // Listener should be added only once per whole entries collection.
-                    // Should we simplify the model and get rid of all futures?
-                    curFut0.listen(lsnr);
+                    if (futs[b.partId] != curFut0) {
+                        opFut.add(curFut0);
+
+                        if (res == null)
+                            res = new ArrayList<>();
+
+                        res.add(curFut0);
+
+                        futs[b.partId] = curFut0;
+                    }
 
                     if (b.batchTopVer == null)
                         b.batchTopVer = topVer;
@@ -1426,14 +1440,28 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     }
                 }
 
-                if (res == null)
-                    res = new ArrayList<>();
+                if (!allowOverwrite() && !topVer.equals(curBatchTopVer)) {
+                    for (int i = 0; i < stripes.length; i++) {
+                        PerStripeBuffer b0 = stripes[i];
 
-                res.add(curFut0);
+                        // renew all stale versions
+                        synchronized (b0) {
+                            // Another thread might already renew the batch
+                            AffinityTopologyVersion bTopVer = b0.batchTopVer;
 
-                if (!allowOverwrite() && !topVer.equals(curBatchTopVer)) {
-                    b.renewBatch(remap);
+                            if(bTopVer != null && topVer.compareTo(bTopVer) > 0) {
+                                GridFutureAdapter<Object> bFut = b0.curFut;
+
+                                b0.renewBatch(remap);
 
+                                bFut.onDone(null,
+                                    new IgniteCheckedException("Topology changed during batch preparation " +
+                                        "[batchTopVer=" + bTopVer + ", topVer=" + topVer + "]"));
+                            }
+                        }
+                    }
+
+                    // double check, it's possible that current future was already overwritten on buffer overflow
                     curFut0.onDone(null, new IgniteCheckedException("Topology changed during batch preparation." +
                         "[batchTopVer=" + curBatchTopVer + ", topVer=" + topVer + "]"));
                 }
@@ -2182,4 +2210,17 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             return S.toString(PerStripeBuffer.class, this, super.toString());
         }
     }
+
+    /** */
+    private static final class SilentCompoundFuture<T,R> extends GridCompoundFuture<T,R> {
+       /** {@inheritDoc} */
+        @Override protected void logError(IgniteLogger log, String msg, Throwable e) {
+            // no-op
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void logDebug(IgniteLogger log, String msg) {
+            // no-op
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1faa8dbe/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index 74a8f41..80cf67b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -96,13 +96,13 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> implements Ig
                     onDone(rdc.reduce());
             }
             catch (RuntimeException e) {
-                U.error(null, "Failed to execute compound future reducer: " + this, e);
+                logError(null, "Failed to execute compound future reducer: " + this, e);
 
                 // Exception in reducer is a bug, so we bypass checkComplete here.
                 onDone(e);
             }
             catch (AssertionError e) {
-                U.error(null, "Failed to execute compound future reducer: " + this, e);
+                logError(null, "Failed to execute compound future reducer: " + this, e);
 
                 // Bypass checkComplete because need to rethrow.
                 onDone(e);
@@ -117,25 +117,21 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> implements Ig
         }
         catch (IgniteCheckedException e) {
             if (!ignoreFailure(e)) {
-                if (e instanceof NodeStoppingException) {
-                    IgniteLogger log = logger();
-
-                    if (log != null && log.isDebugEnabled())
-                        log.debug("Failed to execute compound future reducer, node stopped.");
-                }
+                if (e instanceof NodeStoppingException)
+                    logDebug(logger(), "Failed to execute compound future reducer, node stopped.");
                 else
-                    U.error(null, "Failed to execute compound future reducer: " + this, e);
+                    logError(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
             }
         }
         catch (RuntimeException e) {
-            U.error(null, "Failed to execute compound future reducer: " + this, e);
+            logError(null, "Failed to execute compound future reducer: " + this, e);
 
             onDone(e);
         }
         catch (AssertionError e) {
-            U.error(null, "Failed to execute compound future reducer: " + this, e);
+            logError(null, "Failed to execute compound future reducer: " + this, e);
 
             // Bypass checkComplete because need to rethrow.
             onDone(e);
@@ -278,12 +274,12 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> implements Ig
                 onDone(rdc != null ? rdc.reduce() : null);
             }
             catch (RuntimeException e) {
-                U.error(null, "Failed to execute compound future reducer: " + this, e);
+                logError(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
             }
             catch (AssertionError e) {
-                U.error(null, "Failed to execute compound future reducer: " + this, e);
+                logError(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
 
@@ -293,6 +289,24 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> implements Ig
     }
 
     /**
+     * @param log IgniteLogger.
+     * @param msg ShortMessage.
+     * @param e Exception.
+     */
+    protected void logError(IgniteLogger log, String msg, Throwable e) {
+        U.error(log, msg, e);
+    }
+
+    /**
+     * @param log IgniteLogger.
+     * @param msg ShortMessage.
+     */
+    protected void logDebug(IgniteLogger log, String msg) {
+        if (log != null && log.isDebugEnabled())
+            log.debug(msg);
+    }
+
+    /**
      * Returns future at the specified position in this list.
      *
      * @param idx - index index of the element to return

http://git-wip-us.apache.org/repos/asf/ignite/blob/1faa8dbe/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
index 6d3466b..e90f6b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
@@ -18,10 +18,17 @@
 package org.apache.ignite.internal.processors.datastreamer;
 
 import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -31,11 +38,13 @@ import org.apache.ignite.cache.CacheServerNotFoundException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -45,6 +54,7 @@ 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.apache.ignite.transactions.TransactionException;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Logger;
 import org.apache.log4j.SimpleLayout;
@@ -105,6 +115,8 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testNullPointerExceptionUponDataStreamerClosing() throws Exception {
+        cnt = 0;
+
         startGrids(5);
 
         final CyclicBarrier barrier = new CyclicBarrier(2);
@@ -252,6 +264,135 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testAllOperationFinishedBeforeFutureCompletion() throws Exception {
+        cnt = 0;
+
+        Ignite ignite = startGrids(MAX_CACHE_COUNT);
+
+        final IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME);
+        final CountDownLatch latch = new CountDownLatch(1);
+        final AtomicReference<Throwable> ex = new AtomicReference<>();
+
+        Collection<Map.Entry> entries = new ArrayList<>(100);
+
+        for (int i = 0; i < 100; i++)
+            entries.add(new IgniteBiTuple<>(i, "" + i));
+
+        IgniteDataStreamer ldr = ignite.dataStreamer(DEFAULT_CACHE_NAME);
+
+        ldr.addData(entries).listen(new IgniteInClosure<IgniteFuture<?>>() {
+            @Override public void apply(IgniteFuture<?> future) {
+                try {
+                    future.get();
+
+                    for (int i = 0; i < 100; i++)
+                        assertEquals("" + i, cache.get(i));
+                }
+                catch (Throwable e) {
+                    ex.set(e);
+                }
+
+                latch.countDown();
+            }
+        });
+
+        ldr.tryFlush();
+
+        assertTrue(latch.await(5, TimeUnit.SECONDS));
+
+        Throwable e = ex.get();
+
+        if(e != null) {
+            if(e instanceof Error)
+                throw (Error) e;
+
+            if(e instanceof RuntimeException)
+                throw (RuntimeException) e;
+
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemapOnTopologyChangeDuringUpdatePreparation() throws Exception {
+        cnt = 0;
+
+        Ignite ignite = startGrids(MAX_CACHE_COUNT);
+
+        final int threads = 8;
+        final int entries = threads * 10000;
+        final long timeout = 10000;
+
+        final CountDownLatch l1 = new CountDownLatch(threads);
+        final CountDownLatch l2 = new CountDownLatch(1);
+        final AtomicInteger cntr = new AtomicInteger();
+
+        final AtomicReference<Throwable> ex = new AtomicReference<>();
+
+        final IgniteDataStreamer ldr = ignite.dataStreamer(DEFAULT_CACHE_NAME);
+
+        final IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    int i = cntr.getAndIncrement();
+
+                    for (int j = 0; i < (entries >> 1); i += threads) {
+                        ldr.addData(i, i);
+
+                        if(j++ % 1000 == 0)
+                            ldr.tryFlush();
+                    }
+
+                    l1.countDown();
+
+                    assertTrue(l2.await(timeout, TimeUnit.MILLISECONDS));
+
+                    for (int j = 0; i < entries; i += threads) {
+                        ldr.addData(i, i);
+
+                        if(j++ % 1000 == 0)
+                            ldr.tryFlush();
+                    }
+                }
+                catch (Throwable e) {
+                    ex.compareAndSet(null, e);
+                }
+            }
+        }, threads, "loader");
+
+        assertTrue(l1.await(timeout, TimeUnit.MILLISECONDS));
+
+        stopGrid(MAX_CACHE_COUNT - 1);
+
+        l2.countDown();
+
+        fut.get(timeout);
+
+        ldr.close();
+
+        Throwable e = ex.get();
+
+        if(e != null) {
+            if(e instanceof Error)
+                throw (Error) e;
+
+            if(e instanceof RuntimeException)
+                throw (RuntimeException) e;
+
+            throw new RuntimeException(e);
+        }
+
+        IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+        for(int i = 0; i < entries; i++)
+            assertEquals(i, cache.get(i));
+    }
+
+    /**
      * Cluster topology mismatch shall result in DataStreamer retrying cache update with the latest topology and
      * no error logged to the console.
      *
@@ -366,4 +507,4 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
             super.sendMessage(node, msg, ackC);
         }
     }
-}
\ No newline at end of file
+}


[33/50] [abbrv] ignite git commit: IGNITE-6480 fix serverTopologyNodes method - Fixes #2726.

Posted by yz...@apache.org.
IGNITE-6480 fix serverTopologyNodes method - Fixes #2726.

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/1ccdc978
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1ccdc978
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1ccdc978

Branch: refs/heads/ignite-2.1.5-p1
Commit: 1ccdc97898b37c7f2240cb222c98899603536fc0
Parents: 24f904e
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Fri Sep 22 13:14:57 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Sep 22 13:24:34 2017 +0300

----------------------------------------------------------------------
 .../internal/managers/discovery/GridDiscoveryManager.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1ccdc978/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 18815ac..5ce7816 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
@@ -183,7 +183,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     private static final int DISCOVERY_HISTORY_SIZE = getInteger(IGNITE_DISCOVERY_HISTORY_SIZE, 500);
 
     /** Predicate filtering out daemon nodes. */
-    private static final IgnitePredicate<ClusterNode> FILTER_DAEMON = new P1<ClusterNode>() {
+    private static final IgnitePredicate<ClusterNode> FILTER_NOT_DAEMON = new P1<ClusterNode>() {
         @Override public boolean apply(ClusterNode n) {
             return !n.isDaemon();
         }
@@ -738,7 +738,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     discoEvt.eventNode(node);
                     discoEvt.type(EVT_NODE_JOINED);
 
-                    discoEvt.topologySnapshot(topVer, new ArrayList<>(F.view(topSnapshot, FILTER_DAEMON)));
+                    discoEvt.topologySnapshot(topVer, new ArrayList<>(F.view(topSnapshot, FILTER_NOT_DAEMON)));
 
                     discoWrk.discoCache = discoCache;
 
@@ -2059,7 +2059,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Server topology nodes or {@code null} if there are no nodes for passed in version.
      */
     @Nullable public Collection<ClusterNode> serverTopologyNodes(long topVer) {
-        return F.view(topology(topVer), F.not(FILTER_CLI));
+        return F.view(topology(topVer), F.not(FILTER_CLI), FILTER_NOT_DAEMON);
     }
 
     /** @return All daemon nodes in topology. */
@@ -2522,7 +2522,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 evt.node(ctx.discovery().localNode());
                 evt.eventNode(node);
                 evt.type(type);
-                evt.topologySnapshot(topVer, U.<ClusterNode, ClusterNode>arrayList(topSnapshot, FILTER_DAEMON));
+                evt.topologySnapshot(topVer, U.<ClusterNode, ClusterNode>arrayList(topSnapshot, FILTER_NOT_DAEMON));
 
                 if (type == EVT_NODE_METRICS_UPDATED)
                     evt.message("Metrics were updated: " + node);


[12/50] [abbrv] ignite git commit: IGNITE-6449 Added missing cache configuration properties. (cherry picked from commit 9b2bca7)

Posted by yz...@apache.org.
IGNITE-6449 Added missing cache configuration properties.
(cherry picked from commit 9b2bca7)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 1a6461582eac95e1e51faf8acd9b8bd10394dd47
Parents: 4e38b79
Author: vsisko <vs...@gridgain.com>
Authored: Wed Sep 20 19:07:55 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 20 19:09:00 2017 +0700

----------------------------------------------------------------------
 .../commands/cache/VisorCacheCommand.scala      | 23 ++++++++++++++++++--
 1 file changed, 21 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1a646158/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 42ff01c..f754275 100755
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -839,11 +839,24 @@ object VisorCacheCommand {
 
         cacheT += ("Group", cfg.getGroupName)
         cacheT += ("Dynamic Deployment ID", cfg.getDynamicDeploymentId)
+        cacheT += ("System", bool2Str(cfg.isSystem))
+
         cacheT += ("Mode", cfg.getMode)
         cacheT += ("Atomicity Mode", safe(cfg.getAtomicityMode))
         cacheT += ("Statistic Enabled", bool2Str(cfg.isStatisticsEnabled))
         cacheT += ("Management Enabled", bool2Str(cfg.isManagementEnabled))
 
+        cacheT += ("On-heap cache enabled", bool2Str(cfg.isOnheapCacheEnabled))
+        cacheT += ("Partition Loss Policy", cfg.getPartitionLossPolicy)
+        cacheT += ("Query Parallelism", cfg.getQueryParallelism)
+        cacheT += ("Copy On Read", bool2Str(cfg.isCopyOnRead))
+        cacheT += ("Listener Configurations", cfg.getListenerConfigurations)
+        cacheT += ("Load Previous Value", bool2Str(cfg.isLoadPreviousValue))
+        cacheT += ("Memory Policy Name", cfg.getMemoryPolicyName)
+        cacheT += ("Node Filter", cfg.getNodeFilter)
+        cacheT += ("Read From Backup", bool2Str(cfg.isReadFromBackup))
+        cacheT += ("Topology Validator", cfg.getTopologyValidator)
+
         cacheT += ("Time To Live Eager Flag", cfg.isEagerTtl)
 
         cacheT += ("Write Synchronization Mode", safe(cfg.getWriteSynchronizationMode))
@@ -860,6 +873,8 @@ object VisorCacheCommand {
         cacheT += ("Rebalance Timeout", rebalanceCfg.getTimeout)
         cacheT += ("Rebalance Delay", rebalanceCfg.getPartitionedDelay)
         cacheT += ("Time Between Rebalance Messages", rebalanceCfg.getThrottle)
+        cacheT += ("Rebalance Batches Count", rebalanceCfg.getBatchesPrefetchCnt)
+        cacheT += ("Rebalance Cache Order", rebalanceCfg.getRebalanceOrder)
 
         cacheT += ("Eviction Policy Enabled", bool2Str(evictCfg.getPolicy != null))
         cacheT += ("Eviction Policy", safe(evictCfg.getPolicy))
@@ -881,8 +896,9 @@ object VisorCacheCommand {
         cacheT += ("Store Keep Binary", storeCfg.isStoreKeepBinary)
         cacheT += ("Store Read Through", bool2Str(storeCfg.isReadThrough))
         cacheT += ("Store Write Through", bool2Str(storeCfg.isWriteThrough))
+        cacheT += ("Store Write Coalescing", bool2Str(storeCfg.getWriteBehindCoalescing))
 
-        cacheT += ("Write-Behind Enabled", bool2Str(storeCfg.isEnabled))
+        cacheT += ("Write-Behind Enabled", bool2Str(storeCfg.isWriteBehindEnabled))
         cacheT += ("Write-Behind Flush Size", storeCfg.getFlushSize)
         cacheT += ("Write-Behind Frequency", storeCfg.getFlushFrequency)
         cacheT += ("Write-Behind Flush Threads Count", storeCfg.getFlushThreadCount)
@@ -895,8 +911,11 @@ object VisorCacheCommand {
         cacheT += ("Expiry Policy Factory Class Name", safe(cfg.getExpiryPolicyFactory))
 
         cacheT +=("Query Execution Time Threshold", queryCfg.getLongQueryWarningTimeout)
-        cacheT +=("Query Schema Name", queryCfg.getSqlSchema)
         cacheT +=("Query Escaped Names", bool2Str(queryCfg.isSqlEscapeAll))
+        cacheT +=("Query Schema Name", queryCfg.getSqlSchema)
+        cacheT +=("Query Indexed Types", queryCfg.getIndexedTypes)
+        cacheT +=("Maximum payload size for offheap indexes", cfg.getSqlIndexMaxInlineSize)
+        cacheT +=("Query Metrics History Size", cfg.getQueryDetailMetricsSize)
 
         val sqlFxs = queryCfg.getSqlFunctionClasses
 


[47/50] [abbrv] ignite git commit: IGNITE-6050 Fix eternal wait in DataStreamerTest.TestBufferSize

Posted by yz...@apache.org.
IGNITE-6050 Fix eternal wait in DataStreamerTest.TestBufferSize

This closes #2448

(cherry picked from commit 2cf1260)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 251a333dfbf09d3f6b69bbb2b7658eb3d88fcf0a
Parents: a318c4a
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Aug 29 14:16:00 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Sep 25 13:33:13 2017 +0300

----------------------------------------------------------------------
 .../Dataload/DataStreamerTest.cs                | 71 +++++++++++++-------
 1 file changed, 48 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/251a333d/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
index de95003..fe5955f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
@@ -38,6 +38,9 @@ namespace Apache.Ignite.Core.Tests.Dataload
         /** Node. */
         private IIgnite _grid;
 
+        /** Node 2. */
+        private IIgnite _grid2;
+
         /** Cache. */
         private ICache<int, int?> _cache;
 
@@ -49,7 +52,7 @@ namespace Apache.Ignite.Core.Tests.Dataload
         {
             _grid = Ignition.Start(TestUtils.GetTestConfiguration());
 
-            Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            _grid2 = Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration())
             {
                 IgniteInstanceName = "grid1"
             });
@@ -228,42 +231,64 @@ namespace Apache.Ignite.Core.Tests.Dataload
         [Test]
         public void TestBufferSize()
         {
-            using (IDataStreamer<int, int> ldr = _grid.GetDataStreamer<int, int>(CacheName))
+            using (var ldr = _grid.GetDataStreamer<int, int>(CacheName))
             {
-                var fut = ldr.AddData(1, 1);
+                const int timeout = 5000;
+
+                var part1 = GetPrimaryPartitionKeys(_grid, 4);
+                var part2 = GetPrimaryPartitionKeys(_grid2, 4);
+
+                var task = ldr.AddData(part1[0], part1[0]);
 
                 Thread.Sleep(100);
 
-                Assert.IsFalse(fut.IsCompleted);
+                Assert.IsFalse(task.IsCompleted);
 
                 ldr.PerNodeBufferSize = 2;
 
-                ldr.AddData(2, 2);
-                ldr.AddData(3, 3);
-                ldr.AddData(4, 4).Wait();
-                fut.Wait();
+                ldr.AddData(part2[0], part2[0]);
+                ldr.AddData(part1[1], part1[1]);
+                Assert.IsTrue(ldr.AddData(part2[1], part2[1]).Wait(timeout));
+                Assert.IsTrue(task.Wait(timeout));
 
-                Assert.AreEqual(1, _cache.Get(1));
-                Assert.AreEqual(2, _cache.Get(2));
-                Assert.AreEqual(3, _cache.Get(3));
-                Assert.AreEqual(4, _cache.Get(4));
+                Assert.AreEqual(part1[0], _cache.Get(part1[0]));
+                Assert.AreEqual(part1[1], _cache.Get(part1[1]));
+                Assert.AreEqual(part2[0], _cache.Get(part2[0]));
+                Assert.AreEqual(part2[1], _cache.Get(part2[1]));
 
-                ldr.AddData(new List<KeyValuePair<int, int>>
+                Assert.IsTrue(ldr.AddData(new[]
                 {
-                    new KeyValuePair<int, int>(5, 5), 
-                    new KeyValuePair<int, int>(6, 6),
-                    new KeyValuePair<int, int>(7, 7), 
-                    new KeyValuePair<int, int>(8, 8)
-                }).Wait();
-
-                Assert.AreEqual(5, _cache.Get(5));
-                Assert.AreEqual(6, _cache.Get(6));
-                Assert.AreEqual(7, _cache.Get(7));
-                Assert.AreEqual(8, _cache.Get(8));
+                    new KeyValuePair<int, int>(part1[2], part1[2]),
+                    new KeyValuePair<int, int>(part1[3], part1[3]),
+                    new KeyValuePair<int, int>(part2[2], part2[2]),
+                    new KeyValuePair<int, int>(part2[3], part2[3])
+                }).Wait(timeout));
+
+                Assert.AreEqual(part1[2], _cache.Get(part1[2]));
+                Assert.AreEqual(part1[3], _cache.Get(part1[3]));
+                Assert.AreEqual(part2[2], _cache.Get(part2[2]));
+                Assert.AreEqual(part2[3], _cache.Get(part2[3]));
             }
         }
 
         /// <summary>
+        /// Gets the primary partition keys.
+        /// </summary>
+        private static int[] GetPrimaryPartitionKeys(IIgnite ignite, int count)
+        {
+            var affinity = ignite.GetAffinity(CacheName);
+            
+            var localNode = ignite.GetCluster().GetLocalNode();
+
+            var part = affinity.GetPrimaryPartitions(localNode).First();
+
+            return Enumerable.Range(0, int.MaxValue)
+                .Where(k => affinity.GetPartition(k) == part)
+                .Take(count)
+                .ToArray();
+        }
+
+        /// <summary>
         /// Test close.
         /// </summary>
         [Test]


[11/50] [abbrv] ignite git commit: IGNITE-6435 Web Console: Added version to footer. (cherry picked from commit 3c75184)

Posted by yz...@apache.org.
IGNITE-6435 Web Console: Added version to footer.
(cherry picked from commit 3c75184)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 4e38b792517ff182391b78821937db57353a7388
Parents: 668e29f
Author: Dmitriy Shabalin <ds...@gridgain.com>
Authored: Wed Sep 20 19:00:09 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 20 19:01:58 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/app.js         |   2 +
 .../app/components/version-picker/index.js      |   2 +-
 .../app/modules/branding/branding.module.js     |   4 +-
 .../app/modules/branding/branding.provider.js   | 111 ------------
 .../app/modules/branding/branding.service.js    |  47 +++++
 .../modules/configuration/Version.service.js    | 169 ------------------
 .../configuration/configuration.module.js       |   4 -
 .../generator/ConfigurationGenerator.js         |   2 +-
 .../configuration/generator/Maven.service.js    |   2 +-
 .../frontend/app/services/Version.service.js    | 171 +++++++++++++++++++
 .../frontend/app/services/Version.spec.js       | 106 ++++++++++++
 .../web-console/frontend/app/services/index.js  |  23 +++
 .../frontend/test/unit/Version.test.js          | 106 ------------
 13 files changed, 354 insertions(+), 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/app.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js
index f072fc5..f2ee8ef 100644
--- a/modules/web-console/frontend/app/app.js
+++ b/modules/web-console/frontend/app/app.js
@@ -128,6 +128,7 @@ import gridItemSelected from './components/grid-item-selected';
 import bsSelectMenu from './components/bs-select-menu';
 import protectFromBsSelectRender from './components/protect-from-bs-select-render';
 import uiGridHovering from './components/ui-grid-hovering';
+import igniteServices from './services';
 
 // Inject external modules.
 import IgniteModules from 'IgniteModules/index';
@@ -188,6 +189,7 @@ angular.module('ignite-console', [
     webConsoleHeader.name,
     webConsoleFooter.name,
     igniteIcon.name,
+    igniteServices.name,
     versionPicker.name,
     userNotifications.name,
     pageConfigure.name,

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/components/version-picker/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/version-picker/index.js b/modules/web-console/frontend/app/components/version-picker/index.js
index f58753e..04ff903 100644
--- a/modules/web-console/frontend/app/components/version-picker/index.js
+++ b/modules/web-console/frontend/app/components/version-picker/index.js
@@ -20,6 +20,6 @@ import component from './component';
 
 export default angular
     .module('ignite-console.version-picker', [
-        'ignite-console.configuration'
+        'ignite-console.services'
     ])
     .component('versionPicker', component);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/branding/branding.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/branding/branding.module.js b/modules/web-console/frontend/app/modules/branding/branding.module.js
index 9c4a5e5..2313728 100644
--- a/modules/web-console/frontend/app/modules/branding/branding.module.js
+++ b/modules/web-console/frontend/app/modules/branding/branding.module.js
@@ -17,7 +17,7 @@
 
 import angular from 'angular';
 
-import IgniteBranding from './branding.provider';
+import IgniteBranding from './branding.service';
 
 import igniteHeaderLogo from './header-logo.directive';
 import igniteHeaderTitle from './header-title.directive';
@@ -30,7 +30,7 @@ angular
 .module('ignite-console.branding', [
     'tf.metatags'
 ])
-.provider(...IgniteBranding)
+.service('IgniteBranding', IgniteBranding)
 .config(['tfMetaTagsProvider', (tfMetaTagsProvider) => {
     tfMetaTagsProvider.setDefaults({
         title: 'Apache Ignite - Management Tool and Configuration Wizard',

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/branding/branding.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/branding/branding.provider.js b/modules/web-console/frontend/app/modules/branding/branding.provider.js
deleted file mode 100644
index 280325e..0000000
--- a/modules/web-console/frontend/app/modules/branding/branding.provider.js
+++ /dev/null
@@ -1,111 +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.
- */
-
-export default ['IgniteBranding', [function() {
-    let titleSuffix = ' – Apache Ignite Web Console';
-
-    let headerLogo = '/images/ignite-logo.svg';
-
-    let headerText = 'Management console for Apache Ignite';
-
-    let showIgniteLogo = false;
-
-    let footerHtml = [
-        '<p>Apache Ignite Web Console</p>',
-        '<p>© 2017 The Apache Software Foundation.</p>',
-        '<p>Apache, Apache Ignite, the Apache feather and the Apache Ignite logo are trademarks of The Apache Software Foundation.</p>'
-    ];
-
-    let termsState;
-
-    let featuresHtml = [
-        '<p>Web Console is an interactive management tool which allows to:</p>',
-        '<ul>',
-        '   <li>Create and download cluster configurations</li>',
-        '   <li>Automatically import domain model from any RDBMS</li>',
-        '   <li>Connect to cluster and run SQL analytics on it</li>',
-        '</ul>'
-    ];
-
-    /**
-     * Change title suffix.
-     *
-     * @param {String} suffix.
-     */
-    this.titleSuffix = (suffix) => {
-        titleSuffix = suffix;
-    };
-
-    /**
-     * Change logo in header.
-     *
-     * @param {String} url Logo path.
-     */
-    this.headerLogo = (url) => {
-        headerLogo = url;
-
-        showIgniteLogo = true;
-    };
-
-    /**
-     * Change text in header.
-     *
-     * @param {String} text Header text.
-     */
-    this.headerText = (text) => {
-        headerText = text;
-    };
-
-    /**
-     * Change text in features.
-     *
-     * @param {Array.<String>} rows Features text.
-     */
-    this.featuresHtml = (rows) => {
-        featuresHtml = rows;
-    };
-
-    /**
-     * Change text in footer.
-     *
-     * @param {Array.<String>} rows Footer text.
-     */
-    this.footerHtml = (rows) => {
-        footerHtml = rows;
-    };
-
-    /**
-     * Set terms and conditions stage.
-     *
-     * @param {String} state
-     */
-    this.termsState = (state) => {
-        termsState = state;
-    };
-
-    this.$get = [() => {
-        return {
-            titleSuffix,
-            headerLogo,
-            headerText,
-            featuresHtml: featuresHtml.join('\n'),
-            footerHtml: footerHtml.join('\n'),
-            showIgniteLogo,
-            termsState
-        };
-    }];
-}]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/branding/branding.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/branding/branding.service.js b/modules/web-console/frontend/app/modules/branding/branding.service.js
new file mode 100644
index 0000000..46bc358
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/branding/branding.service.js
@@ -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.
+ */
+
+export default class {
+    static $inject = ['IgniteVersion'];
+
+    constructor(Version) {
+        this.titleSuffix = ' – Apache Ignite Web Console';
+
+        this.headerLogo = '/images/ignite-logo.svg';
+
+        this.headerText = 'Management console for Apache Ignite';
+
+        this.showIgniteLogo = false;
+
+        this.footerHtml = [
+            `<p>Apache Ignite Web Console (${Version.webConsole})</p>`,
+            '<p>© 2017 The Apache Software Foundation.</p>',
+            '<p>Apache, Apache Ignite, the Apache feather and the Apache Ignite logo are trademarks of The Apache Software Foundation.</p>'
+        ].join('\n');
+
+        this.termsState = null;
+
+        this.featuresHtml = [
+            '<p>Web Console is an interactive management tool which allows to:</p>',
+            '<ul>',
+            '   <li>Create and download cluster configurations</li>',
+            '   <li>Automatically import domain model from any RDBMS</li>',
+            '   <li>Connect to cluster and run SQL analytics on it</li>',
+            '</ul>'
+        ].join('\n');
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/configuration/Version.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js
deleted file mode 100644
index 210b34a..0000000
--- a/modules/web-console/frontend/app/modules/configuration/Version.service.js
+++ /dev/null
@@ -1,169 +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.
- */
-
-import { BehaviorSubject } from 'rxjs/BehaviorSubject';
-
-/**
- * Utility service for version parsing and comparing
- */
-const VERSION_MATCHER = /(\d+)\.(\d+)\.(\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT)?)?(-(\d+))?(-([\da-f]+))?/i;
-
-/**
- * Tries to parse product version from it's string representation.
- *
- * @param {String} ver - String representation of version.
- * @returns {{major: Number, minor: Number, maintenance: Number, stage: String, revTs: Number, revHash: String}} - Object that contains product version fields.
- */
-const parse = (ver) => {
-    // Development or built from source ZIP.
-    ver = ver.replace(/(-DEV|-n\/a)$/i, '');
-
-    const [, major, minor, maintenance, stage, ...chunks] = ver.match(VERSION_MATCHER);
-
-    return {
-        major: parseInt(major, 10),
-        minor: parseInt(minor, 10),
-        maintenance: parseInt(maintenance, 10),
-        stage: (stage || '').substring(1),
-        revTs: chunks[2] ? parseInt(chunks[3], 10) : 0,
-        revHash: chunks[4] ? chunks[5] : null
-    };
-};
-
-const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0;
-
-/**
- * Compare to version.
- * @param a {Object} first compared version.
- * @param b {Object} second compared version.
- * @returns {Number} 1 if a > b, 0 if versions equals, -1 if a < b
- */
-const compare = (a, b) => {
-    let res = numberComparator(a.major, b.major);
-
-    if (res !== 0)
-        return res;
-
-    res = numberComparator(a.minor, b.minor);
-
-    if (res !== 0)
-        return res;
-
-    res = numberComparator(a.maintenance, b.maintenance);
-
-    if (res !== 0)
-        return res;
-
-    return numberComparator(a.stage, b.stage);
-};
-
-export default class IgniteVersion {
-    constructor() {
-        this.supportedVersions = [
-            {
-                label: 'Ignite 2.1',
-                ignite: '2.1.0'
-            },
-            {
-                label: 'Ignite 2.0',
-                ignite: '2.0.0'
-            },
-            {
-                label: 'Ignite 1.x',
-                ignite: '1.9.0'
-            }
-        ];
-
-        /** Current product version. */
-        let current = _.head(this.supportedVersions);
-
-        try {
-            const ignite = localStorage.configurationVersion;
-
-            const restored = _.find(this.supportedVersions, {ignite});
-
-            if (restored)
-                current = restored;
-        }
-        catch (ignored) {
-            // No-op.
-        }
-
-        this.currentSbj = new BehaviorSubject(current);
-
-        this.currentSbj.subscribe({
-            next: (ver) => {
-                try {
-                    localStorage.setItem('configurationVersion', ver.ignite);
-                }
-                catch (ignored) {
-                    // No-op.
-                }
-            }
-        });
-    }
-
-    /**
-     * @return {String} Current Ignite version.
-     */
-    get current() {
-        return this.currentSbj.getValue().ignite;
-    }
-
-    /**
-     * Check if version in range.
-     *
-     * @param {String} target Target version.
-     * @param {String | Array.<String>} ranges Version ranges to compare with.
-     * @returns {Boolean} `True` if version is equal or greater than specified range.
-     */
-    since(target, ...ranges) {
-        const targetVer = parse(target);
-
-        return !!_.find(ranges, (range) => {
-            if (_.isArray(range)) {
-                const [after, before] = range;
-
-                return compare(targetVer, parse(after)) >= 0 &&
-                    (_.isNil(before) || compare(targetVer, parse(before)) < 0);
-            }
-
-            return compare(targetVer, parse(range)) >= 0;
-        });
-    }
-
-    /**
-     * Check whether version before than specified version.
-     *
-     * @param {String} target Target version.
-     * @param {String} ranges Version ranges to compare with.
-     * @return {Boolean} `True` if version before than specified version.
-     */
-    before(target, ...ranges) {
-        return !this.since(target, ...ranges);
-    }
-
-    /**
-     * Check if current version in specified range.
-     *
-     * @param {String|Array.<String>} ranges Version ranges to compare with.
-     * @returns {Boolean} `True` if configuration version is equal or greater than specified range.
-     */
-    available(...ranges) {
-        return this.since(this.current, ...ranges);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/configuration/configuration.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
index 54c6bf1..a350871 100644
--- a/modules/web-console/frontend/app/modules/configuration/configuration.module.js
+++ b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
@@ -17,9 +17,6 @@
 
 import angular from 'angular';
 
-
-import IgniteVersion from './Version.service';
-
 import IgniteClusterDefaults from './generator/defaults/Cluster.service';
 import IgniteClusterPlatformDefaults from './generator/defaults/Cluster.platform.service';
 import IgniteCacheDefaults from './generator/defaults/Cache.service';
@@ -50,7 +47,6 @@ angular
 .service('SpringTransformer', () => IgniteSpringTransformer)
 .service('JavaTransformer', () => IgniteJavaTransformer)
 .service('IgniteSharpTransformer', SharpTransformer)
-.service('IgniteVersion', IgniteVersion)
 .service('IgniteEventGroups', IgniteEventGroups)
 .service('IgniteClusterDefaults', IgniteClusterDefaults)
 .service('IgniteClusterPlatformDefaults', IgniteClusterPlatformDefaults)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index 428a8d0..9d7887a 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -25,7 +25,7 @@ import IgniteCacheDefaults from './defaults/Cache.service';
 import IgniteIGFSDefaults from './defaults/IGFS.service';
 
 import JavaTypes from '../../../services/JavaTypes.service';
-import VersionService from 'app/modules/configuration/Version.service';
+import VersionService from 'app/services/Version.service';
 
 const clusterDflts = new IgniteClusterDefaults();
 const cacheDflts = new IgniteCacheDefaults();

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
index 9bf0a34..a145f2e 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
@@ -16,7 +16,7 @@
  */
 
 import StringBuilder from './StringBuilder';
-import VersionService from 'app/modules/configuration/Version.service';
+import VersionService from 'app/services/Version.service';
 
 const versionService = new VersionService();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/services/Version.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Version.service.js b/modules/web-console/frontend/app/services/Version.service.js
new file mode 100644
index 0000000..8ae23d3
--- /dev/null
+++ b/modules/web-console/frontend/app/services/Version.service.js
@@ -0,0 +1,171 @@
+/*
+ * 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.
+ */
+
+import { BehaviorSubject } from 'rxjs/BehaviorSubject';
+
+/**
+ * Utility service for version parsing and comparing
+ */
+const VERSION_MATCHER = /(\d+)\.(\d+)\.(\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT)?)?(-(\d+))?(-([\da-f]+))?/i;
+
+/**
+ * Tries to parse product version from it's string representation.
+ *
+ * @param {String} ver - String representation of version.
+ * @returns {{major: Number, minor: Number, maintenance: Number, stage: String, revTs: Number, revHash: String}} - Object that contains product version fields.
+ */
+const parse = (ver) => {
+    // Development or built from source ZIP.
+    ver = ver.replace(/(-DEV|-n\/a)$/i, '');
+
+    const [, major, minor, maintenance, stage, ...chunks] = ver.match(VERSION_MATCHER);
+
+    return {
+        major: parseInt(major, 10),
+        minor: parseInt(minor, 10),
+        maintenance: parseInt(maintenance, 10),
+        stage: (stage || '').substring(1),
+        revTs: chunks[2] ? parseInt(chunks[3], 10) : 0,
+        revHash: chunks[4] ? chunks[5] : null
+    };
+};
+
+const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0;
+
+/**
+ * Compare to version.
+ * @param a {Object} first compared version.
+ * @param b {Object} second compared version.
+ * @returns {Number} 1 if a > b, 0 if versions equals, -1 if a < b
+ */
+const compare = (a, b) => {
+    let res = numberComparator(a.major, b.major);
+
+    if (res !== 0)
+        return res;
+
+    res = numberComparator(a.minor, b.minor);
+
+    if (res !== 0)
+        return res;
+
+    res = numberComparator(a.maintenance, b.maintenance);
+
+    if (res !== 0)
+        return res;
+
+    return numberComparator(a.stage, b.stage);
+};
+
+export default class IgniteVersion {
+    constructor() {
+        this.webConsole = '2.1.0';
+
+        this.supportedVersions = [
+            {
+                label: 'Ignite 2.1',
+                ignite: '2.1.0'
+            },
+            {
+                label: 'Ignite 2.0',
+                ignite: '2.0.0'
+            },
+            {
+                label: 'Ignite 1.x',
+                ignite: '1.9.0'
+            }
+        ];
+
+        /** Current product version. */
+        let current = _.head(this.supportedVersions);
+
+        try {
+            const ignite = localStorage.configurationVersion;
+
+            const restored = _.find(this.supportedVersions, {ignite});
+
+            if (restored)
+                current = restored;
+        }
+        catch (ignored) {
+            // No-op.
+        }
+
+        this.currentSbj = new BehaviorSubject(current);
+
+        this.currentSbj.subscribe({
+            next: (ver) => {
+                try {
+                    localStorage.setItem('configurationVersion', ver.ignite);
+                }
+                catch (ignored) {
+                    // No-op.
+                }
+            }
+        });
+    }
+
+    /**
+     * @return {String} Current Ignite version.
+     */
+    get current() {
+        return this.currentSbj.getValue().ignite;
+    }
+
+    /**
+     * Check if version in range.
+     *
+     * @param {String} target Target version.
+     * @param {String | Array.<String>} ranges Version ranges to compare with.
+     * @returns {Boolean} `True` if version is equal or greater than specified range.
+     */
+    since(target, ...ranges) {
+        const targetVer = parse(target);
+
+        return !!_.find(ranges, (range) => {
+            if (_.isArray(range)) {
+                const [after, before] = range;
+
+                return compare(targetVer, parse(after)) >= 0 &&
+                    (_.isNil(before) || compare(targetVer, parse(before)) < 0);
+            }
+
+            return compare(targetVer, parse(range)) >= 0;
+        });
+    }
+
+    /**
+     * Check whether version before than specified version.
+     *
+     * @param {String} target Target version.
+     * @param {String} ranges Version ranges to compare with.
+     * @return {Boolean} `True` if version before than specified version.
+     */
+    before(target, ...ranges) {
+        return !this.since(target, ...ranges);
+    }
+
+    /**
+     * Check if current version in specified range.
+     *
+     * @param {String|Array.<String>} ranges Version ranges to compare with.
+     * @returns {Boolean} `True` if configuration version is equal or greater than specified range.
+     */
+    available(...ranges) {
+        return this.since(this.current, ...ranges);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/services/Version.spec.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Version.spec.js b/modules/web-console/frontend/app/services/Version.spec.js
new file mode 100644
index 0000000..12e769c
--- /dev/null
+++ b/modules/web-console/frontend/app/services/Version.spec.js
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import VersionService from './Version.service';
+
+const INSTANCE = new VersionService();
+
+import { suite, test } from 'mocha';
+import { assert } from 'chai';
+
+suite('VersionServiceTestsSuite', () => {
+    test.skip('Parse 1.7.0-SNAPSHOT', () => {
+        const version = INSTANCE.parse('1.7.0-SNAPSHOT');
+        assert.equal(version.major, 1);
+        assert.equal(version.minor, 7);
+        assert.equal(version.maintenance, 0);
+        assert.equal(version.stage, 'SNAPSHOT');
+        assert.equal(version.revTs, 0);
+        assert.isNull(version.revHash);
+    });
+
+    test.skip('Parse strip -DEV 1.7.0-DEV', () => {
+        const version = INSTANCE.parse('1.7.0-DEV');
+        assert.equal(version.major, 1);
+        assert.equal(version.minor, 7);
+        assert.equal(version.maintenance, 0);
+        assert.equal(version.stage, '');
+    });
+
+    test.skip('Parse strip -n/a 1.7.0-n/a', () => {
+        const version = INSTANCE.parse('1.7.0-n/a');
+        assert.equal(version.major, 1);
+        assert.equal(version.minor, 7);
+        assert.equal(version.maintenance, 0);
+        assert.equal(version.stage, '');
+    });
+
+    test.skip('Check patch version', () => {
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.7.2'), INSTANCE.parse('1.7.1')), 1);
+    });
+
+    test.skip('Check minor version', () => {
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.8.1'), INSTANCE.parse('1.7.1')), 1);
+    });
+
+    test.skip('Check major version', () => {
+        assert.equal(INSTANCE.compare(INSTANCE.parse('2.7.1'), INSTANCE.parse('1.7.1')), 1);
+    });
+
+    test.skip('Version a > b', () => {
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.7.0'), INSTANCE.parse('1.5.0')), 1);
+    });
+
+    test.skip('Version a = b', () => {
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.0.0'), INSTANCE.parse('1.0.0')), 0);
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.0'), INSTANCE.parse('1.2.0')), 0);
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.3'), INSTANCE.parse('1.2.3')), 0);
+
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.0.0-1'), INSTANCE.parse('1.0.0-1')), 0);
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.0-1'), INSTANCE.parse('1.2.0-1')), 0);
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.3-1'), INSTANCE.parse('1.2.3-1')), 0);
+    });
+
+    test.skip('Version a < b', () => {
+        assert.equal(INSTANCE.compare(INSTANCE.parse('1.5.1'), INSTANCE.parse('1.5.2')), -1);
+    });
+
+    test('Check since call', () => {
+        assert.equal(INSTANCE.since('1.5.0', '1.5.0'), true);
+        assert.equal(INSTANCE.since('1.6.0', '1.5.0'), true);
+        assert.equal(INSTANCE.since('1.5.4', ['1.5.5', '1.6.0'], ['1.6.2']), false);
+        assert.equal(INSTANCE.since('1.5.5', ['1.5.5', '1.6.0'], ['1.6.2']), true);
+        assert.equal(INSTANCE.since('1.5.11', ['1.5.5', '1.6.0'], ['1.6.2']), true);
+        assert.equal(INSTANCE.since('1.6.0', ['1.5.5', '1.6.0'], ['1.6.2']), false);
+        assert.equal(INSTANCE.since('1.6.1', ['1.5.5', '1.6.0'], '1.6.2'), false);
+        assert.equal(INSTANCE.since('1.6.2', ['1.5.5', '1.6.0'], ['1.6.2']), true);
+        assert.equal(INSTANCE.since('1.6.3', ['1.5.5', '1.6.0'], '1.6.2'), true);
+    });
+
+    test('Check wrong since call', () => {
+        assert.equal(INSTANCE.since('1.3.0', '1.5.0'), false);
+    });
+
+    test('Check before call', () => {
+        assert.equal(INSTANCE.before('1.5.0', '1.5.0'), false);
+        assert.equal(INSTANCE.before('1.5.0', '1.6.0'), true);
+    });
+
+    test('Check wrong before call', () => {
+        assert.equal(INSTANCE.before('1.5.0', '1.3.0'), false);
+    });
+});

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/app/services/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/index.js b/modules/web-console/frontend/app/services/index.js
new file mode 100644
index 0000000..49e7632
--- /dev/null
+++ b/modules/web-console/frontend/app/services/index.js
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+import angular from 'angular';
+import IgniteVersion from './Version.service';
+
+export default angular
+    .module('ignite-console.services', [])
+    .service('IgniteVersion', IgniteVersion);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e38b792/modules/web-console/frontend/test/unit/Version.test.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/test/unit/Version.test.js b/modules/web-console/frontend/test/unit/Version.test.js
deleted file mode 100644
index b0ca3d9..0000000
--- a/modules/web-console/frontend/test/unit/Version.test.js
+++ /dev/null
@@ -1,106 +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.
- */
-
-import VersionService from '../../app/modules/configuration/Version.service';
-
-const INSTANCE = new VersionService();
-
-import { suite, test } from 'mocha';
-import { assert } from 'chai';
-
-suite('VersionServiceTestsSuite', () => {
-    test.skip('Parse 1.7.0-SNAPSHOT', () => {
-        const version = INSTANCE.parse('1.7.0-SNAPSHOT');
-        assert.equal(version.major, 1);
-        assert.equal(version.minor, 7);
-        assert.equal(version.maintenance, 0);
-        assert.equal(version.stage, 'SNAPSHOT');
-        assert.equal(version.revTs, 0);
-        assert.isNull(version.revHash);
-    });
-
-    test.skip('Parse strip -DEV 1.7.0-DEV', () => {
-        const version = INSTANCE.parse('1.7.0-DEV');
-        assert.equal(version.major, 1);
-        assert.equal(version.minor, 7);
-        assert.equal(version.maintenance, 0);
-        assert.equal(version.stage, '');
-    });
-
-    test.skip('Parse strip -n/a 1.7.0-n/a', () => {
-        const version = INSTANCE.parse('1.7.0-n/a');
-        assert.equal(version.major, 1);
-        assert.equal(version.minor, 7);
-        assert.equal(version.maintenance, 0);
-        assert.equal(version.stage, '');
-    });
-
-    test.skip('Check patch version', () => {
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.7.2'), INSTANCE.parse('1.7.1')), 1);
-    });
-
-    test.skip('Check minor version', () => {
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.8.1'), INSTANCE.parse('1.7.1')), 1);
-    });
-
-    test.skip('Check major version', () => {
-        assert.equal(INSTANCE.compare(INSTANCE.parse('2.7.1'), INSTANCE.parse('1.7.1')), 1);
-    });
-
-    test.skip('Version a > b', () => {
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.7.0'), INSTANCE.parse('1.5.0')), 1);
-    });
-
-    test.skip('Version a = b', () => {
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.0.0'), INSTANCE.parse('1.0.0')), 0);
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.0'), INSTANCE.parse('1.2.0')), 0);
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.3'), INSTANCE.parse('1.2.3')), 0);
-
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.0.0-1'), INSTANCE.parse('1.0.0-1')), 0);
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.0-1'), INSTANCE.parse('1.2.0-1')), 0);
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.2.3-1'), INSTANCE.parse('1.2.3-1')), 0);
-    });
-
-    test.skip('Version a < b', () => {
-        assert.equal(INSTANCE.compare(INSTANCE.parse('1.5.1'), INSTANCE.parse('1.5.2')), -1);
-    });
-
-    test('Check since call', () => {
-        assert.equal(INSTANCE.since('1.5.0', '1.5.0'), true);
-        assert.equal(INSTANCE.since('1.6.0', '1.5.0'), true);
-        assert.equal(INSTANCE.since('1.5.4', ['1.5.5', '1.6.0'], ['1.6.2']), false);
-        assert.equal(INSTANCE.since('1.5.5', ['1.5.5', '1.6.0'], ['1.6.2']), true);
-        assert.equal(INSTANCE.since('1.5.11', ['1.5.5', '1.6.0'], ['1.6.2']), true);
-        assert.equal(INSTANCE.since('1.6.0', ['1.5.5', '1.6.0'], ['1.6.2']), false);
-        assert.equal(INSTANCE.since('1.6.1', ['1.5.5', '1.6.0'], '1.6.2'), false);
-        assert.equal(INSTANCE.since('1.6.2', ['1.5.5', '1.6.0'], ['1.6.2']), true);
-        assert.equal(INSTANCE.since('1.6.3', ['1.5.5', '1.6.0'], '1.6.2'), true);
-    });
-
-    test('Check wrong since call', () => {
-        assert.equal(INSTANCE.since('1.3.0', '1.5.0'), false);
-    });
-
-    test('Check before call', () => {
-        assert.equal(INSTANCE.before('1.5.0', '1.5.0'), false);
-        assert.equal(INSTANCE.before('1.5.0', '1.6.0'), true);
-    });
-
-    test('Check wrong before call', () => {
-        assert.equal(INSTANCE.before('1.5.0', '1.3.0'), false);
-    });
-});


[05/50] [abbrv] ignite git commit: IGNITE-6099: ODBC: Implemented SQLGetInfo for all info types. This closes #2689.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/718e365f/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index 4925957..362d46b 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -48,184 +48,846 @@ namespace ignite
             {
                 switch (type)
                 {
+#ifdef SQL_ACCESSIBLE_PROCEDURES
+                    DBG_STR_CASE(SQL_ACCESSIBLE_PROCEDURES);
+#endif // SQL_ACCESSIBLE_PROCEDURES
+#ifdef SQL_ACCESSIBLE_TABLES
+                    DBG_STR_CASE(SQL_ACCESSIBLE_TABLES);
+#endif // SQL_ACCESSIBLE_TABLES
+#ifdef SQL_ACTIVE_ENVIRONMENTS
+                    DBG_STR_CASE(SQL_ACTIVE_ENVIRONMENTS);
+#endif // SQL_ACTIVE_ENVIRONMENTS
+#ifdef SQL_DRIVER_NAME
                     DBG_STR_CASE(SQL_DRIVER_NAME);
+#endif // SQL_DRIVER_NAME
+#ifdef SQL_DBMS_NAME
                     DBG_STR_CASE(SQL_DBMS_NAME);
+#endif // SQL_DBMS_NAME
+#ifdef SQL_DRIVER_ODBC_VER
                     DBG_STR_CASE(SQL_DRIVER_ODBC_VER);
+#endif // SQL_DRIVER_ODBC_VER
+#ifdef SQL_DBMS_VER
                     DBG_STR_CASE(SQL_DBMS_VER);
+#endif // SQL_DBMS_VER
+#ifdef SQL_DRIVER_VER
                     DBG_STR_CASE(SQL_DRIVER_VER);
+#endif // SQL_DRIVER_VER
+#ifdef SQL_COLUMN_ALIAS
                     DBG_STR_CASE(SQL_COLUMN_ALIAS);
+#endif // SQL_COLUMN_ALIAS
+#ifdef SQL_IDENTIFIER_QUOTE_CHAR
                     DBG_STR_CASE(SQL_IDENTIFIER_QUOTE_CHAR);
+#endif // SQL_IDENTIFIER_QUOTE_CHAR
+#ifdef SQL_CATALOG_NAME_SEPARATOR
                     DBG_STR_CASE(SQL_CATALOG_NAME_SEPARATOR);
+#endif // SQL_CATALOG_NAME_SEPARATOR
+#ifdef SQL_SPECIAL_CHARACTERS
                     DBG_STR_CASE(SQL_SPECIAL_CHARACTERS);
+#endif // SQL_SPECIAL_CHARACTERS
+#ifdef SQL_CATALOG_TERM
                     DBG_STR_CASE(SQL_CATALOG_TERM);
+#endif // SQL_CATALOG_TERM
+#ifdef SQL_TABLE_TERM
                     DBG_STR_CASE(SQL_TABLE_TERM);
+#endif // SQL_TABLE_TERM
+#ifdef SQL_SCHEMA_TERM
                     DBG_STR_CASE(SQL_SCHEMA_TERM);
+#endif // SQL_SCHEMA_TERM
+#ifdef SQL_NEED_LONG_DATA_LEN
                     DBG_STR_CASE(SQL_NEED_LONG_DATA_LEN);
-//                    DBG_STR_CASE(SQL_ASYNC_DBC_FUNCTIONS);
+#endif // SQL_NEED_LONG_DATA_LEN
+#ifdef SQL_ASYNC_DBC_FUNCTIONS
+                    DBG_STR_CASE(SQL_ASYNC_DBC_FUNCTIONS);
+#endif // SQL_ASYNC_DBC_FUNCTIONS
+#ifdef SQL_ASYNC_NOTIFICATION
                     DBG_STR_CASE(SQL_ASYNC_NOTIFICATION);
+#endif // SQL_ASYNC_NOTIFICATION
+#ifdef SQL_GETDATA_EXTENSIONS
                     DBG_STR_CASE(SQL_GETDATA_EXTENSIONS);
+#endif // SQL_GETDATA_EXTENSIONS
+#ifdef SQL_ODBC_INTERFACE_CONFORMANCE
                     DBG_STR_CASE(SQL_ODBC_INTERFACE_CONFORMANCE);
+#endif // SQL_ODBC_INTERFACE_CONFORMANCE
+#ifdef SQL_SQL_CONFORMANCE
                     DBG_STR_CASE(SQL_SQL_CONFORMANCE);
+#endif // SQL_SQL_CONFORMANCE
+#ifdef SQL_CATALOG_USAGE
                     DBG_STR_CASE(SQL_CATALOG_USAGE);
+#endif // SQL_CATALOG_USAGE
+#ifdef SQL_SCHEMA_USAGE
                     DBG_STR_CASE(SQL_SCHEMA_USAGE);
+#endif // SQL_SCHEMA_USAGE
+#ifdef SQL_MAX_IDENTIFIER_LEN
                     DBG_STR_CASE(SQL_MAX_IDENTIFIER_LEN);
+#endif // SQL_MAX_IDENTIFIER_LEN
+#ifdef SQL_AGGREGATE_FUNCTIONS
                     DBG_STR_CASE(SQL_AGGREGATE_FUNCTIONS);
+#endif // SQL_AGGREGATE_FUNCTIONS
+#ifdef SQL_NUMERIC_FUNCTIONS
                     DBG_STR_CASE(SQL_NUMERIC_FUNCTIONS);
+#endif // SQL_NUMERIC_FUNCTIONS
+#ifdef SQL_STRING_FUNCTIONS
                     DBG_STR_CASE(SQL_STRING_FUNCTIONS);
+#endif // SQL_STRING_FUNCTIONS
+#ifdef SQL_TIMEDATE_FUNCTIONS
                     DBG_STR_CASE(SQL_TIMEDATE_FUNCTIONS);
+#endif // SQL_TIMEDATE_FUNCTIONS
+#ifdef SQL_TIMEDATE_ADD_INTERVALS
                     DBG_STR_CASE(SQL_TIMEDATE_ADD_INTERVALS);
+#endif // SQL_TIMEDATE_ADD_INTERVALS
+#ifdef SQL_TIMEDATE_DIFF_INTERVALS
                     DBG_STR_CASE(SQL_TIMEDATE_DIFF_INTERVALS);
+#endif // SQL_TIMEDATE_DIFF_INTERVALS
+#ifdef SQL_DATETIME_LITERALS
                     DBG_STR_CASE(SQL_DATETIME_LITERALS);
+#endif // SQL_DATETIME_LITERALS
+#ifdef SQL_SYSTEM_FUNCTIONS
                     DBG_STR_CASE(SQL_SYSTEM_FUNCTIONS);
+#endif // SQL_SYSTEM_FUNCTIONS
+#ifdef SQL_CONVERT_FUNCTIONS
                     DBG_STR_CASE(SQL_CONVERT_FUNCTIONS);
+#endif // SQL_CONVERT_FUNCTIONS
+#ifdef SQL_OJ_CAPABILITIES
                     DBG_STR_CASE(SQL_OJ_CAPABILITIES);
+#endif // SQL_OJ_CAPABILITIES
+#ifdef SQL_POS_OPERATIONS
                     DBG_STR_CASE(SQL_POS_OPERATIONS);
+#endif // SQL_POS_OPERATIONS
+#ifdef SQL_MAX_CONCURRENT_ACTIVITIES
                     DBG_STR_CASE(SQL_MAX_CONCURRENT_ACTIVITIES);
+#endif // SQL_MAX_CONCURRENT_ACTIVITIES
+#ifdef SQL_CURSOR_COMMIT_BEHAVIOR
                     DBG_STR_CASE(SQL_CURSOR_COMMIT_BEHAVIOR);
+#endif // SQL_CURSOR_COMMIT_BEHAVIOR
+#ifdef SQL_CURSOR_ROLLBACK_BEHAVIOR
                     DBG_STR_CASE(SQL_CURSOR_ROLLBACK_BEHAVIOR);
+#endif // SQL_CURSOR_ROLLBACK_BEHAVIOR
+#ifdef SQL_TXN_CAPABLE
                     DBG_STR_CASE(SQL_TXN_CAPABLE);
+#endif // SQL_TXN_CAPABLE
+#ifdef SQL_QUOTED_IDENTIFIER_CASE
                     DBG_STR_CASE(SQL_QUOTED_IDENTIFIER_CASE);
+#endif // SQL_QUOTED_IDENTIFIER_CASE
+#ifdef SQL_SQL92_NUMERIC_VALUE_FUNCTIONS
                     DBG_STR_CASE(SQL_SQL92_NUMERIC_VALUE_FUNCTIONS);
+#endif // SQL_SQL92_NUMERIC_VALUE_FUNCTIONS
+#ifdef SQL_SQL92_STRING_FUNCTIONS
                     DBG_STR_CASE(SQL_SQL92_STRING_FUNCTIONS);
+#endif // SQL_SQL92_STRING_FUNCTIONS
+#ifdef SQL_SQL92_DATETIME_FUNCTIONS
                     DBG_STR_CASE(SQL_SQL92_DATETIME_FUNCTIONS);
+#endif // SQL_SQL92_DATETIME_FUNCTIONS
+#ifdef SQL_SQL92_PREDICATES
                     DBG_STR_CASE(SQL_SQL92_PREDICATES);
+#endif // SQL_SQL92_PREDICATES
+#ifdef SQL_SQL92_RELATIONAL_JOIN_OPERATORS
                     DBG_STR_CASE(SQL_SQL92_RELATIONAL_JOIN_OPERATORS);
+#endif // SQL_SQL92_RELATIONAL_JOIN_OPERATORS
+#ifdef SQL_SQL92_VALUE_EXPRESSIONS
                     DBG_STR_CASE(SQL_SQL92_VALUE_EXPRESSIONS);
+#endif // SQL_SQL92_VALUE_EXPRESSIONS
+#ifdef SQL_STATIC_CURSOR_ATTRIBUTES1
                     DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES1);
+#endif // SQL_STATIC_CURSOR_ATTRIBUTES1
+#ifdef SQL_STATIC_CURSOR_ATTRIBUTES2
                     DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES2);
+#endif // SQL_STATIC_CURSOR_ATTRIBUTES2
+#ifdef SQL_CONVERT_BIGINT
                     DBG_STR_CASE(SQL_CONVERT_BIGINT);
+#endif // SQL_CONVERT_BIGINT
+#ifdef SQL_CONVERT_BINARY
                     DBG_STR_CASE(SQL_CONVERT_BINARY);
+#endif // SQL_CONVERT_BINARY
+#ifdef SQL_CONVERT_BIT
                     DBG_STR_CASE(SQL_CONVERT_BIT);
+#endif // SQL_CONVERT_BIT
+#ifdef SQL_CONVERT_CHAR
                     DBG_STR_CASE(SQL_CONVERT_CHAR);
+#endif // SQL_CONVERT_CHAR
+#ifdef SQL_CONVERT_DATE
                     DBG_STR_CASE(SQL_CONVERT_DATE);
+#endif // SQL_CONVERT_DATE
+#ifdef SQL_CONVERT_DECIMAL
                     DBG_STR_CASE(SQL_CONVERT_DECIMAL);
+#endif // SQL_CONVERT_DECIMAL
+#ifdef SQL_CONVERT_DOUBLE
                     DBG_STR_CASE(SQL_CONVERT_DOUBLE);
+#endif // SQL_CONVERT_DOUBLE
+#ifdef SQL_CONVERT_FLOAT
                     DBG_STR_CASE(SQL_CONVERT_FLOAT);
+#endif // SQL_CONVERT_FLOAT
+#ifdef SQL_CONVERT_INTEGER
                     DBG_STR_CASE(SQL_CONVERT_INTEGER);
+#endif // SQL_CONVERT_INTEGER
+#ifdef SQL_CONVERT_LONGVARCHAR
                     DBG_STR_CASE(SQL_CONVERT_LONGVARCHAR);
+#endif // SQL_CONVERT_LONGVARCHAR
+#ifdef SQL_CONVERT_NUMERIC
                     DBG_STR_CASE(SQL_CONVERT_NUMERIC);
+#endif // SQL_CONVERT_NUMERIC
+#ifdef SQL_CONVERT_REAL
                     DBG_STR_CASE(SQL_CONVERT_REAL);
+#endif // SQL_CONVERT_REAL
+#ifdef SQL_CONVERT_SMALLINT
                     DBG_STR_CASE(SQL_CONVERT_SMALLINT);
+#endif // SQL_CONVERT_SMALLINT
+#ifdef SQL_CONVERT_TIME
                     DBG_STR_CASE(SQL_CONVERT_TIME);
+#endif // SQL_CONVERT_TIME
+#ifdef SQL_CONVERT_TIMESTAMP
                     DBG_STR_CASE(SQL_CONVERT_TIMESTAMP);
+#endif // SQL_CONVERT_TIMESTAMP
+#ifdef SQL_CONVERT_TINYINT
                     DBG_STR_CASE(SQL_CONVERT_TINYINT);
+#endif // SQL_CONVERT_TINYINT
+#ifdef SQL_CONVERT_VARBINARY
                     DBG_STR_CASE(SQL_CONVERT_VARBINARY);
+#endif // SQL_CONVERT_VARBINARY
+#ifdef SQL_CONVERT_VARCHAR
                     DBG_STR_CASE(SQL_CONVERT_VARCHAR);
+#endif // SQL_CONVERT_VARCHAR
+#ifdef SQL_CONVERT_LONGVARBINARY
                     DBG_STR_CASE(SQL_CONVERT_LONGVARBINARY);
+#endif // SQL_CONVERT_LONGVARBINARY
+#ifdef SQL_CONVERT_WCHAR
                     DBG_STR_CASE(SQL_CONVERT_WCHAR);
+#endif // SQL_CONVERT_WCHAR
+#ifdef SQL_CONVERT_INTERVAL_DAY_TIME
                     DBG_STR_CASE(SQL_CONVERT_INTERVAL_DAY_TIME);
+#endif // SQL_CONVERT_INTERVAL_DAY_TIME
+#ifdef SQL_CONVERT_INTERVAL_YEAR_MONTH
                     DBG_STR_CASE(SQL_CONVERT_INTERVAL_YEAR_MONTH);
+#endif // SQL_CONVERT_INTERVAL_YEAR_MONTH
+#ifdef SQL_CONVERT_WLONGVARCHAR
                     DBG_STR_CASE(SQL_CONVERT_WLONGVARCHAR);
+#endif // SQL_CONVERT_WLONGVARCHAR
+#ifdef SQL_CONVERT_WVARCHAR
                     DBG_STR_CASE(SQL_CONVERT_WVARCHAR);
+#endif // SQL_CONVERT_WVARCHAR
+#ifdef SQL_CONVERT_GUID
                     DBG_STR_CASE(SQL_CONVERT_GUID);
+#endif // SQL_CONVERT_GUID
+#ifdef SQL_SCROLL_OPTIONS
                     DBG_STR_CASE(SQL_SCROLL_OPTIONS);
+#endif // SQL_SCROLL_OPTIONS
+#ifdef SQL_PARAM_ARRAY_ROW_COUNTS
                     DBG_STR_CASE(SQL_PARAM_ARRAY_ROW_COUNTS);
+#endif // SQL_PARAM_ARRAY_ROW_COUNTS
+#ifdef SQL_PARAM_ARRAY_SELECTS
                     DBG_STR_CASE(SQL_PARAM_ARRAY_SELECTS);
-                default:
-                    break;
+#endif // SQL_PARAM_ARRAY_SELECTS
+#ifdef SQL_ALTER_DOMAIN
+                    DBG_STR_CASE(SQL_ALTER_DOMAIN);
+#endif // SQL_ALTER_DOMAIN
+#ifdef SQL_ASYNC_MODE
+                    DBG_STR_CASE(SQL_ASYNC_MODE);
+#endif // SQL_ASYNC_MODE
+#ifdef SQL_BATCH_ROW_COUNT
+                    DBG_STR_CASE(SQL_BATCH_ROW_COUNT);
+#endif // SQL_BATCH_ROW_COUNT
+#ifdef SQL_BATCH_SUPPORT
+                    DBG_STR_CASE(SQL_BATCH_SUPPORT);
+#endif // SQL_BATCH_SUPPORT
+#ifdef SQL_BOOKMARK_PERSISTENCE
+                    DBG_STR_CASE(SQL_BOOKMARK_PERSISTENCE);
+#endif // SQL_BOOKMARK_PERSISTENCE
+#ifdef SQL_CATALOG_LOCATION
+                    DBG_STR_CASE(SQL_CATALOG_LOCATION);
+#endif // SQL_CATALOG_LOCATION
+#ifdef SQL_CATALOG_NAME
+                    DBG_STR_CASE(SQL_CATALOG_NAME);
+#endif // SQL_CATALOG_NAME
+#ifdef SQL_COLLATION_SEQ
+                    DBG_STR_CASE(SQL_COLLATION_SEQ);
+#endif // SQL_COLLATION_SEQ
+#ifdef SQL_CONCAT_NULL_BEHAVIOR
+                    DBG_STR_CASE(SQL_CONCAT_NULL_BEHAVIOR);
+#endif // SQL_CONCAT_NULL_BEHAVIOR
+#ifdef SQL_CORRELATION_NAME
+                    DBG_STR_CASE(SQL_CORRELATION_NAME);
+#endif // SQL_CORRELATION_NAME
+#ifdef SQL_CREATE_ASSERTION
+                    DBG_STR_CASE(SQL_CREATE_ASSERTION);
+#endif // SQL_CREATE_ASSERTION
+#ifdef SQL_CREATE_CHARACTER_SET
+                    DBG_STR_CASE(SQL_CREATE_CHARACTER_SET);
+#endif // SQL_CREATE_CHARACTER_SET
+#ifdef SQL_CREATE_COLLATION
+                    DBG_STR_CASE(SQL_CREATE_COLLATION);
+#endif // SQL_CREATE_COLLATION
+#ifdef SQL_CREATE_DOMAIN
+                    DBG_STR_CASE(SQL_CREATE_DOMAIN);
+#endif // SQL_CREATE_DOMAIN
+#ifdef SQL_CREATE_TABLE
+                    DBG_STR_CASE(SQL_CREATE_TABLE);
+#endif // SQL_CREATE_TABLE
+#ifdef SQL_CREATE_TRANSLATION
+                    DBG_STR_CASE(SQL_CREATE_TRANSLATION);
+#endif // SQL_CREATE_TRANSLATION
+#ifdef SQL_CREATE_VIEW
+                    DBG_STR_CASE(SQL_CREATE_VIEW);
+#endif // SQL_CREATE_VIEW
+#ifdef SQL_CURSOR_SENSITIVITY
+                    DBG_STR_CASE(SQL_CURSOR_SENSITIVITY);
+#endif // SQL_CURSOR_SENSITIVITY
+#ifdef SQL_DATA_SOURCE_NAME
+                    DBG_STR_CASE(SQL_DATA_SOURCE_NAME);
+#endif // SQL_DATA_SOURCE_NAME
+#ifdef SQL_DATA_SOURCE_READ_ONLY
+                    DBG_STR_CASE(SQL_DATA_SOURCE_READ_ONLY);
+#endif // SQL_DATA_SOURCE_READ_ONLY
+#ifdef SQL_DATABASE_NAME
+                    DBG_STR_CASE(SQL_DATABASE_NAME);
+#endif // SQL_DATABASE_NAME
+#ifdef SQL_DDL_INDEX
+                    DBG_STR_CASE(SQL_DDL_INDEX);
+#endif // SQL_DDL_INDEX
+#ifdef SQL_DEFAULT_TXN_ISOLATION
+                    DBG_STR_CASE(SQL_DEFAULT_TXN_ISOLATION);
+#endif // SQL_DEFAULT_TXN_ISOLATION
+#ifdef SQL_DESCRIBE_PARAMETER
+                    DBG_STR_CASE(SQL_DESCRIBE_PARAMETER);
+#endif // SQL_DESCRIBE_PARAMETER
+#ifdef SQL_DROP_ASSERTION
+                    DBG_STR_CASE(SQL_DROP_ASSERTION);
+#endif // SQL_DROP_ASSERTION
+#ifdef SQL_DROP_CHARACTER_SET
+                    DBG_STR_CASE(SQL_DROP_CHARACTER_SET);
+#endif // SQL_DROP_CHARACTER_SET
+#ifdef SQL_DROP_COLLATION
+                    DBG_STR_CASE(SQL_DROP_COLLATION);
+#endif // SQL_DROP_COLLATION
+#ifdef SQL_DROP_DOMAIN
+                    DBG_STR_CASE(SQL_DROP_DOMAIN);
+#endif // SQL_DROP_DOMAIN
+#ifdef SQL_DROP_SCHEMA
+                    DBG_STR_CASE(SQL_DROP_SCHEMA);
+#endif // SQL_DROP_SCHEMA
+#ifdef SQL_DROP_TABLE
+                    DBG_STR_CASE(SQL_DROP_TABLE);
+#endif // SQL_DROP_TABLE
+#ifdef SQL_DROP_TRANSLATION
+                    DBG_STR_CASE(SQL_DROP_TRANSLATION);
+#endif // SQL_DROP_TRANSLATION
+#ifdef SQL_DROP_VIEW
+                    DBG_STR_CASE(SQL_DROP_VIEW);
+#endif // SQL_DROP_VIEW
+#ifdef SQL_DYNAMIC_CURSOR_ATTRIBUTES1
+                    DBG_STR_CASE(SQL_DYNAMIC_CURSOR_ATTRIBUTES1);
+#endif // SQL_DYNAMIC_CURSOR_ATTRIBUTES1
+#ifdef SQL_DYNAMIC_CURSOR_ATTRIBUTES2
+                    DBG_STR_CASE(SQL_DYNAMIC_CURSOR_ATTRIBUTES2);
+#endif // SQL_DYNAMIC_CURSOR_ATTRIBUTES2
+#ifdef SQL_EXPRESSIONS_IN_ORDERBY
+                    DBG_STR_CASE(SQL_EXPRESSIONS_IN_ORDERBY);
+#endif // SQL_EXPRESSIONS_IN_ORDERBY
+#ifdef SQL_FILE_USAGE
+                    DBG_STR_CASE(SQL_FILE_USAGE);
+#endif // SQL_FILE_USAGE
+#ifdef SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES1
+                    DBG_STR_CASE(SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES1);
+#endif // SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES1
+#ifdef SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES2
+                    DBG_STR_CASE(SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES2);
+#endif // SQL_FORWARD_ONLY_CURSOR_ATTRIBUTES2
+#ifdef SQL_GROUP_BY
+                    DBG_STR_CASE(SQL_GROUP_BY);
+#endif // SQL_GROUP_BY
+#ifdef SQL_IDENTIFIER_CASE
+                    DBG_STR_CASE(SQL_IDENTIFIER_CASE);
+#endif // SQL_IDENTIFIER_CASE
+#ifdef SQL_INDEX_KEYWORDS
+                    DBG_STR_CASE(SQL_INDEX_KEYWORDS);
+#endif // SQL_INDEX_KEYWORDS
+#ifdef SQL_INFO_SCHEMA_VIEWS
+                    DBG_STR_CASE(SQL_INFO_SCHEMA_VIEWS);
+#endif // SQL_INFO_SCHEMA_VIEWS
+#ifdef SQL_INSERT_STATEMENT
+                    DBG_STR_CASE(SQL_INSERT_STATEMENT);
+#endif // SQL_INSERT_STATEMENT
+#ifdef SQL_INTEGRITY
+                    DBG_STR_CASE(SQL_INTEGRITY);
+#endif // SQL_INTEGRITY
+#ifdef SQL_KEYSET_CURSOR_ATTRIBUTES1
+                    DBG_STR_CASE(SQL_KEYSET_CURSOR_ATTRIBUTES1);
+#endif // SQL_KEYSET_CURSOR_ATTRIBUTES1
+#ifdef SQL_KEYSET_CURSOR_ATTRIBUTES2
+                    DBG_STR_CASE(SQL_KEYSET_CURSOR_ATTRIBUTES2);
+#endif // SQL_KEYSET_CURSOR_ATTRIBUTES2
+#ifdef SQL_KEYWORDS
+                    DBG_STR_CASE(SQL_KEYWORDS);
+#endif // SQL_KEYWORDS
+#ifdef SQL_LIKE_ESCAPE_CLAUSE
+                    DBG_STR_CASE(SQL_LIKE_ESCAPE_CLAUSE);
+#endif // SQL_LIKE_ESCAPE_CLAUSE
+#ifdef SQL_MAX_ASYNC_CONCURRENT_STATEMENTS
+                    DBG_STR_CASE(SQL_MAX_ASYNC_CONCURRENT_STATEMENTS);
+#endif // SQL_MAX_ASYNC_CONCURRENT_STATEMENTS
+#ifdef SQL_MAX_BINARY_LITERAL_LEN
+                    DBG_STR_CASE(SQL_MAX_BINARY_LITERAL_LEN);
+#endif // SQL_MAX_BINARY_LITERAL_LEN
+#ifdef SQL_MAX_CATALOG_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_CATALOG_NAME_LEN);
+#endif // SQL_MAX_CATALOG_NAME_LEN
+#ifdef SQL_MAX_CHAR_LITERAL_LEN
+                    DBG_STR_CASE(SQL_MAX_CHAR_LITERAL_LEN);
+#endif // SQL_MAX_CHAR_LITERAL_LEN
+#ifdef SQL_MAX_COLUMN_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_COLUMN_NAME_LEN);
+#endif // SQL_MAX_COLUMN_NAME_LEN
+#ifdef SQL_MAX_COLUMNS_IN_GROUP_BY
+                    DBG_STR_CASE(SQL_MAX_COLUMNS_IN_GROUP_BY);
+#endif // SQL_MAX_COLUMNS_IN_GROUP_BY
+#ifdef SQL_MAX_COLUMNS_IN_INDEX
+                    DBG_STR_CASE(SQL_MAX_COLUMNS_IN_INDEX);
+#endif // SQL_MAX_COLUMNS_IN_INDEX
+#ifdef SQL_MAX_COLUMNS_IN_ORDER_BY
+                    DBG_STR_CASE(SQL_MAX_COLUMNS_IN_ORDER_BY);
+#endif // SQL_MAX_COLUMNS_IN_ORDER_BY
+#ifdef SQL_MAX_COLUMNS_IN_SELECT
+                    DBG_STR_CASE(SQL_MAX_COLUMNS_IN_SELECT);
+#endif // SQL_MAX_COLUMNS_IN_SELECT
+#ifdef SQL_MAX_COLUMNS_IN_TABLE
+                    DBG_STR_CASE(SQL_MAX_COLUMNS_IN_TABLE);
+#endif // SQL_MAX_COLUMNS_IN_TABLE
+#ifdef SQL_MAX_CURSOR_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_CURSOR_NAME_LEN);
+#endif // SQL_MAX_CURSOR_NAME_LEN
+#ifdef SQL_MAX_DRIVER_CONNECTIONS
+                    DBG_STR_CASE(SQL_MAX_DRIVER_CONNECTIONS);
+#endif // SQL_MAX_DRIVER_CONNECTIONS
+#ifdef SQL_MAX_INDEX_SIZE
+                    DBG_STR_CASE(SQL_MAX_INDEX_SIZE);
+#endif // SQL_MAX_INDEX_SIZE
+#ifdef SQL_MAX_PROCEDURE_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_PROCEDURE_NAME_LEN);
+#endif // SQL_MAX_PROCEDURE_NAME_LEN
+#ifdef SQL_MAX_ROW_SIZE
+                    DBG_STR_CASE(SQL_MAX_ROW_SIZE);
+#endif // SQL_MAX_ROW_SIZE
+#ifdef SQL_MAX_ROW_SIZE_INCLUDES_LONG
+                    DBG_STR_CASE(SQL_MAX_ROW_SIZE_INCLUDES_LONG);
+#endif // SQL_MAX_ROW_SIZE_INCLUDES_LONG
+#ifdef SQL_MAX_SCHEMA_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_SCHEMA_NAME_LEN);
+#endif // SQL_MAX_SCHEMA_NAME_LEN
+#ifdef SQL_MAX_STATEMENT_LEN
+                    DBG_STR_CASE(SQL_MAX_STATEMENT_LEN);
+#endif // SQL_MAX_STATEMENT_LEN
+#ifdef SQL_MAX_TABLE_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_TABLE_NAME_LEN);
+#endif // SQL_MAX_TABLE_NAME_LEN
+#ifdef SQL_MAX_TABLES_IN_SELECT
+                    DBG_STR_CASE(SQL_MAX_TABLES_IN_SELECT);
+#endif // SQL_MAX_TABLES_IN_SELECT
+#ifdef SQL_MAX_USER_NAME_LEN
+                    DBG_STR_CASE(SQL_MAX_USER_NAME_LEN);
+#endif // SQL_MAX_USER_NAME_LEN
+#ifdef SQL_MULT_RESULT_SETS
+                    DBG_STR_CASE(SQL_MULT_RESULT_SETS);
+#endif // SQL_MULT_RESULT_SETS
+#ifdef SQL_MULTIPLE_ACTIVE_TXN
+                    DBG_STR_CASE(SQL_MULTIPLE_ACTIVE_TXN);
+#endif // SQL_MULTIPLE_ACTIVE_TXN
+#ifdef SQL_NON_NULLABLE_COLUMNS
+                    DBG_STR_CASE(SQL_NON_NULLABLE_COLUMNS);
+#endif // SQL_NON_NULLABLE_COLUMNS
+#ifdef SQL_NULL_COLLATION
+                    DBG_STR_CASE(SQL_NULL_COLLATION);
+#endif // SQL_NULL_COLLATION
+#ifdef SQL_ORDER_BY_COLUMNS_IN_SELECT
+                    DBG_STR_CASE(SQL_ORDER_BY_COLUMNS_IN_SELECT);
+#endif // SQL_ORDER_BY_COLUMNS_IN_SELECT
+#ifdef SQL_PROCEDURE_TERM
+                    DBG_STR_CASE(SQL_PROCEDURE_TERM);
+#endif // SQL_PROCEDURE_TERM
+#ifdef SQL_PROCEDURES
+                    DBG_STR_CASE(SQL_PROCEDURES);
+#endif // SQL_PROCEDURES
+#ifdef SQL_ROW_UPDATES
+                    DBG_STR_CASE(SQL_ROW_UPDATES);
+#endif // SQL_ROW_UPDATES
+#ifdef SQL_SEARCH_PATTERN_ESCAPE
+                    DBG_STR_CASE(SQL_SEARCH_PATTERN_ESCAPE);
+#endif // SQL_SEARCH_PATTERN_ESCAPE
+#ifdef SQL_SERVER_NAME
+                    DBG_STR_CASE(SQL_SERVER_NAME);
+#endif // SQL_SERVER_NAME
+#ifdef SQL_SQL92_FOREIGN_KEY_DELETE_RULE
+                    DBG_STR_CASE(SQL_SQL92_FOREIGN_KEY_DELETE_RULE);
+#endif // SQL_SQL92_FOREIGN_KEY_DELETE_RULE
+#ifdef SQL_SQL92_FOREIGN_KEY_UPDATE_RULE
+                    DBG_STR_CASE(SQL_SQL92_FOREIGN_KEY_UPDATE_RULE);
+#endif // SQL_SQL92_FOREIGN_KEY_UPDATE_RULE
+#ifdef SQL_SQL92_GRANT
+                    DBG_STR_CASE(SQL_SQL92_GRANT);
+#endif // SQL_SQL92_GRANT
+#ifdef SQL_SQL92_REVOKE
+                    DBG_STR_CASE(SQL_SQL92_REVOKE);
+#endif // SQL_SQL92_REVOKE
+#ifdef SQL_SQL92_ROW_VALUE_CONSTRUCTOR
+                    DBG_STR_CASE(SQL_SQL92_ROW_VALUE_CONSTRUCTOR);
+#endif // SQL_SQL92_ROW_VALUE_CONSTRUCTOR
+#ifdef SQL_STANDARD_CLI_CONFORMANCE
+                    DBG_STR_CASE(SQL_STANDARD_CLI_CONFORMANCE);
+#endif // SQL_STANDARD_CLI_CONFORMANCE
+#ifdef SQL_SUBQUERIES
+                    DBG_STR_CASE(SQL_SUBQUERIES);
+#endif // SQL_SUBQUERIES
+#ifdef SQL_TXN_ISOLATION_OPTION
+                    DBG_STR_CASE(SQL_TXN_ISOLATION_OPTION);
+#endif // SQL_TXN_ISOLATION_OPTION
+#ifdef SQL_UNION
+                    DBG_STR_CASE(SQL_UNION);
+#endif // SQL_UNION
+#ifdef SQL_USER_NAME
+                    DBG_STR_CASE(SQL_USER_NAME);
+#endif // SQL_USER_NAME
+#ifdef SQL_ALTER_TABLE
+                    DBG_STR_CASE(SQL_ALTER_TABLE);
+#endif // SQL_ALTER_TABLE
+                    default:
+                        break;
                 }
                 return "<< UNKNOWN TYPE >>";
             }
 
 #undef DBG_STR_CASE
 
-            ConnectionInfo::ConnectionInfo() : strParams(), intParams(),
-                shortParams()
+            ConnectionInfo::ConnectionInfo(const Configuration& config) :
+                strParams(),
+                intParams(),
+                shortParams(),
+                config(config)
             {
+                //
                 //======================= String Params =======================
+                //
+
                 // Driver name.
+#ifdef SQL_DRIVER_NAME
                 strParams[SQL_DRIVER_NAME] = "Apache Ignite";
+#endif // SQL_DRIVER_NAME
+#ifdef SQL_DBMS_NAME
                 strParams[SQL_DBMS_NAME]   = "Apache Ignite";
+#endif // SQL_DBMS_NAME
 
                 // ODBC version.
+#ifdef SQL_DRIVER_ODBC_VER
                 strParams[SQL_DRIVER_ODBC_VER] = "03.00";
-                strParams[SQL_DBMS_VER]        = "03.00";
+#endif // SQL_DRIVER_ODBC_VER
 
 #ifdef SQL_DRIVER_VER
-                // Driver version. At a minimum, the version is of the form
-                // ##.##.####, where the first two digits are the major version,
-                // the next two digits are the minor version, and the last four
-                // digits are the release version.
-                strParams[SQL_DRIVER_VER] = "01.05.0000";
+                // Driver version. At a minimum, the version is of the form ##.##.####, where the first two digits are
+                // the major version, the next two digits are the minor version, and the last four digits are the
+                // release version.
+                strParams[SQL_DRIVER_VER] = "02.03.0000";
 #endif // SQL_DRIVER_VER
+#ifdef SQL_DBMS_VER
+                strParams[SQL_DBMS_VER] = "02.03.0000";
+#endif // SQL_DBMS_VER
 
 #ifdef SQL_COLUMN_ALIAS
-                // A character string: "Y" if the data source supports column
-                // aliases; otherwise, "N".
+                // A character string: "Y" if the data source supports column aliases; otherwise, "N".
                 strParams[SQL_COLUMN_ALIAS] = "Y";
 #endif // SQL_COLUMN_ALIAS
 
 #ifdef SQL_IDENTIFIER_QUOTE_CHAR
-                // The character string that is used as the starting and ending
-                // delimiter of a quoted (delimited) identifier in SQL statements.
-                // Identifiers passed as arguments to ODBC functions do not have to
-                // be quoted. If the data source does not support quoted
-                // identifiers, a blank is returned.
+                // The character string that is used as the starting and ending delimiter of a quoted (delimited)
+                // identifier in SQL statements. Identifiers passed as arguments to ODBC functions do not have to be
+                // quoted. If the data source does not support quoted identifiers, a blank is returned.
                 strParams[SQL_IDENTIFIER_QUOTE_CHAR] = "";
 #endif // SQL_IDENTIFIER_QUOTE_CHAR
 
 #ifdef SQL_CATALOG_NAME_SEPARATOR
-                // A character string: the character or characters that the data
-                // source defines as the separator between a catalog name and the
-                // qualified name element that follows or precedes it.
+                // A character string: the character or characters that the data source defines as the separator between
+                // a catalog name and the qualified name element that follows or precedes it.
                 strParams[SQL_CATALOG_NAME_SEPARATOR] = ".";
 #endif // SQL_CATALOG_NAME_SEPARATOR
 
 #ifdef SQL_SPECIAL_CHARACTERS
-                // A character string that contains all special characters (that
-                // is, all characters except a through z, A through Z, 0 through 9,
-                // and underscore) that can be used in an identifier name, such as
-                // a table name, column name, or index name, on the data source.
+                // A character string that contains all special characters (that is, all characters except a through z,
+                // A through Z, 0 through 9, and underscore) that can be used in an identifier name, such as a table
+                // name, column name, or index name, on the data source.
                 strParams[SQL_SPECIAL_CHARACTERS] = "";
 #endif // SQL_SPECIAL_CHARACTERS
 
 #ifdef SQL_CATALOG_TERM
-                // A character string with the data source vendor's name for
-                // a catalog; for example, "database" or "directory". This string
-                // can be in upper, lower, or mixed case.
-                strParams[SQL_CATALOG_TERM] = "catalog";
+                // A character string with the data source vendor's name for a catalog; for example, "database" or
+                // "directory". This string can be in upper, lower, or mixed case. This InfoType has been renamed for
+                // ODBC 3.0 from the ODBC 2.0 InfoType SQL_QUALIFIER_TERM.
+                strParams[SQL_CATALOG_TERM] = "";
 #endif // SQL_CATALOG_TERM
 
+#ifdef SQL_QUALIFIER_TERM
+                strParams[SQL_QUALIFIER_TERM] = "";
+#endif // SQL_QUALIFIER_TERM
+
 #ifdef SQL_TABLE_TERM
-                // A character string with the data source vendor's name for
-                // a table; for example, "table" or "file".
+                // A character string with the data source vendor's name for a table; for example, "table" or "file".
                 strParams[SQL_TABLE_TERM] = "table";
 #endif // SQL_TABLE_TERM
 
 #ifdef SQL_SCHEMA_TERM
-                // A character string with the data source vendor's name for
-                // a schema; for example, "owner", "Authorization ID", or "Schema".
+                // A character string with the data source vendor's name for a schema; for example, "owner",
+                // "Authorization ID", or "Schema".
                 strParams[SQL_SCHEMA_TERM] = "schema";
 #endif // SQL_SCHEMA_TERM
 
 #ifdef SQL_NEED_LONG_DATA_LEN
-                // A character string: "Y" if the data source needs the length
-                // of a long data value (the data type is SQL_LONGVARCHAR,
-                // SQL_LONGVARBINARY) before that value is sent to the data
-                // source, "N" if it does not.
+                // A character string: "Y" if the data source needs the length of a long data value (the data type is
+                // SQL_LONGVARCHAR, SQL_LONGVARBINARY) before that value is sent to the data source, "N" if it does not.
                 strParams[SQL_NEED_LONG_DATA_LEN ] = "Y";
 #endif // SQL_NEED_LONG_DATA_LEN
 
-#ifdef SQL_ASYNC_DBC_FUNCTIONS
+#ifdef SQL_ACCESSIBLE_PROCEDURES
+                // A character string: "Y" if the user can execute all procedures returned by SQLProcedures; "N" if
+                // there may be procedures returned that the user cannot execute.
+                strParams[SQL_ACCESSIBLE_PROCEDURES] = "Y";
+#endif // SQL_ACCESSIBLE_PROCEDURES
+
+#ifdef SQL_ACCESSIBLE_TABLES
+                // A character string: "Y" if the user is guaranteed SELECT privileges to all tables returned by
+                // SQLTables; "N" if there may be tables returned that the user cannot access.
+                strParams[SQL_ACCESSIBLE_TABLES] = "Y";
+#endif // SQL_ACCESSIBLE_TABLES
+
+#ifdef SQL_CATALOG_NAME
+                // A character string: "Y" if the server supports catalog names, or "N" if it does not.
+                // An SQL - 92 Full level-conformant driver will always return "Y".
+                strParams[SQL_CATALOG_NAME] = "N";
+#endif // SQL_CATALOG_NAME
+
+#ifdef SQL_COLLATION_SEQ
+                // The name of the collation sequence. This is a character string that indicates the name of the default
+                // collation for the default character set for this server (for example, 'ISO 8859-1' or EBCDIC). If
+                // this is unknown, an empty string will be returned. An SQL-92 Full level-conformant driver will always
+                // return a non-empty string.
+                strParams[SQL_COLLATION_SEQ] = "UTF-8";
+#endif // SQL_COLLATION_SEQ
+
+#ifdef SQL_DATA_SOURCE_NAME
+                // A character string with the data source name that was used during connection.
+                //
+                // If the application called SQLConnect, this is the value of the szDSN argument. If the application
+                // called SQLDriverConnect or SQLBrowseConnect, this is the value of the DSN keyword in the connection
+                // string passed to the driver. If the connection string did not contain the DSN keyword (such as when
+                // it contains the DRIVER keyword), this is an empty string.
+                strParams[SQL_DATA_SOURCE_NAME] = config.GetDsn();
+#endif // SQL_DATA_SOURCE_NAME
+
+#ifdef SQL_DATA_SOURCE_READ_ONLY
+                // A character string. "Y" if the data source is set to READ ONLY mode, "N" if it is otherwise.
+                //
+                // This characteristic pertains only to the data source itself; it is not a characteristic of the driver
+                // that enables access to the data source. A driver that is read/write can be used with a data source
+                // that is read-only. If a driver is read-only, all of its data sources must be read-only and must
+                // return SQL_DATA_SOURCE_READ_ONLY.
+                strParams[SQL_DATA_SOURCE_READ_ONLY] = "N";
+#endif // SQL_DATA_SOURCE_READ_ONLY
+
+#ifdef SQL_DATABASE_NAME
+                // A character string with the name of the current database in use, if the data source defines a named
+                // object called "database".
+                strParams[SQL_DATABASE_NAME] = "";
+#endif // SQL_DATABASE_NAME
+
+#ifdef SQL_DESCRIBE_PARAMETER
+                // A character string: "Y" if parameters can be described; "N", if not.
+                // An SQL-92 Full level-conformant driver will usually return "Y" because it will support the DESCRIBE
+                // INPUT statement. Because this does not directly specify the underlying SQL support, however,
+                // describing parameters might not be supported, even in a SQL-92 Full level-conformant driver.
+                strParams[SQL_DESCRIBE_PARAMETER] = "N";
+#endif // SQL_DESCRIBE_PARAMETER
+
+#ifdef SQL_EXPRESSIONS_IN_ORDERBY
+                // A character string: "Y" if the data source supports expressions in the ORDER BY list; "N" if it does
+                // not.
+                strParams[SQL_EXPRESSIONS_IN_ORDERBY] = "Y";
+#endif // SQL_EXPRESSIONS_IN_ORDERBY
+
+#ifdef SQL_INTEGRITY
+                // A character string: "Y" if the data source supports the Integrity Enhancement Facility; "N" if it
+                // does not.
+                strParams[SQL_INTEGRITY] = "N";
+#endif // SQL_INTEGRITY
+
+#ifdef SQL_KEYWORDS
+                // A character string that contains a comma-separated list of all data source-specific keywords. This
+                // list does not contain keywords specific to ODBC or keywords used by both the data source and ODBC.
+                // This list represents all the reserved keywords; interoperable applications should not use these words
+                // in object names.
+                // The #define value SQL_ODBC_KEYWORDS contains a comma - separated list of ODBC keywords.
+                strParams[SQL_KEYWORDS] = "LIMIT,MINUS,OFFSET,ROWNUM,SYSDATE,SYSTIME,SYSTIMESTAMP,TODAY";
+#endif // SQL_KEYWORDS
+
+#ifdef SQL_LIKE_ESCAPE_CLAUSE
+                // A character string: "Y" if the data source supports an escape character for the percent character (%)
+                // and underscore character (_) in a LIKE predicate and the driver supports the ODBC syntax for defining
+                // a LIKE predicate escape character; "N" otherwise.
+                strParams[SQL_LIKE_ESCAPE_CLAUSE] = "N";
+#endif // SQL_LIKE_ESCAPE_CLAUSE
+
+#ifdef SQL_MAX_ROW_SIZE_INCLUDES_LONG
+                // A character string: "Y" if the maximum row size returned for the SQL_MAX_ROW_SIZE information type
+                // includes the length of all SQL_LONGVARCHAR and SQL_LONGVARBINARY columns in the row; "N" otherwise.
+                strParams[SQL_MAX_ROW_SIZE_INCLUDES_LONG] = "Y";
+#endif // SQL_MAX_ROW_SIZE_INCLUDES_LONG
+
+#ifdef SQL_MULT_RESULT_SETS
+                // A character string: "Y" if the data source supports multiple result sets, "N" if it does not.
+                strParams[SQL_MULT_RESULT_SETS] = "N";
+#endif // SQL_MULT_RESULT_SETS
+
+#ifdef SQL_MULTIPLE_ACTIVE_TXN
+                // A character string: "Y" if the driver supports more than one active transaction at the same time,
+                // "N" if only one transaction can be active at any time.
+                strParams[SQL_MULTIPLE_ACTIVE_TXN] = "N";
+#endif // SQL_MULTIPLE_ACTIVE_TXN
+
+#ifdef SQL_ORDER_BY_COLUMNS_IN_SELECT
+                // A character string: "Y" if the columns in the ORDER BY clause must be in the select list;
+                // otherwise, "N".
+                strParams[SQL_ORDER_BY_COLUMNS_IN_SELECT] = "N";
+#endif // SQL_ORDER_BY_COLUMNS_IN_SELECT
+
+#ifdef SQL_PROCEDURE_TERM
+                // A character string with the data source vendor's name for a procedure; for example,
+                // "database procedure", "stored procedure", "procedure", "package", or "stored query".
+                strParams[SQL_PROCEDURE_TERM] = "stored procedure";
+#endif // SQL_PROCEDURE_TERM
+
+#ifdef SQL_PROCEDURE_TERM
+                // A character string: "Y" if the data source supports procedures and the driver supports the ODBC
+                // procedure invocation syntax; "N" otherwise.
+                strParams[SQL_PROCEDURE_TERM] = "N";
+#endif // SQL_PROCEDURE_TERM
+
+#ifdef SQL_ROW_UPDATES
+                // A character string: "Y" if a keyset-driven or mixed cursor maintains row versions or values for all
+                // fetched rows and therefore can detect any updates that were made to a row by any user since the row
+                // was last fetched. (This applies only to updates, not to deletions or insertions.) The driver can
+                // return the SQL_ROW_UPDATED flag to the row status array when SQLFetchScroll is called. Otherwise, "N"
+                strParams[SQL_ROW_UPDATES] = "N";
+#endif // SQL_ROW_UPDATES
+
+#ifdef SQL_SEARCH_PATTERN_ESCAPE
+                // A character string specifying what the driver supports as an escape character that allows the use of
+                // the pattern match metacharacters underscore (_) and percent sign (%) as valid characters in search
+                // patterns. This escape character applies only for those catalog function arguments that support search
+                // strings. If this string is empty, the driver does not support a search-pattern escape character.
+                // Because this information type does not indicate general support of the escape character in the LIKE
+                // predicate, SQL-92 does not include requirements for this character string.
+                // This InfoType is limited to catalog functions. For a description of the use of the escape character
+                // in search pattern strings, see Pattern Value Arguments.
+                strParams[SQL_SEARCH_PATTERN_ESCAPE] = "\\";
+#endif // SQL_SEARCH_PATTERN_ESCAPE
+
+#ifdef SQL_SERVER_NAME
+                // A character string with the actual data source-specific server name; useful when a data source name
+                // is used during SQLConnect, SQLDriverConnect, and SQLBrowseConnect.
+                strParams[SQL_SERVER_NAME] = "Apache Ignite";
+#endif // SQL_SERVER_NAME
+
+#ifdef SQL_USER_NAME
+                // A character string with the name used in a particular database, which can be different from the login
+                // name.
+                strParams[SQL_USER_NAME] = "apache_ignite_user";
+#endif // SQL_USER_NAME
+
+                //
                 //====================== Integer Params =======================
-                // Indicates if the driver can execute functions asynchronously
-                // on the connection handle.
-                // SQL_ASYNC_DBC_CAPABLE = The driver can execute connection
-                // functions asynchronously.
-                // SQL_ASYNC_DBC_NOT_CAPABLE = The driver can not execute
-                // connection functions asynchronously.
+                //
+
+#ifdef SQL_ASYNC_DBC_FUNCTIONS
+                // Indicates if the driver can execute functions asynchronously on the connection handle.
+                // SQL_ASYNC_DBC_CAPABLE = The driver can execute connection functions asynchronously.
+                // SQL_ASYNC_DBC_NOT_CAPABLE = The driver can not execute connection functions asynchronously.
                 intParams[SQL_ASYNC_DBC_FUNCTIONS] = SQL_ASYNC_DBC_NOT_CAPABLE;
 #endif // SQL_ASYNC_DBC_FUNCTIONS
 
+#ifdef SQL_ASYNC_MODE
+                // Indicates the level of asynchronous support in the driver:
+                // SQL_AM_CONNECTION = Connection level asynchronous execution is supported.Either all statement handles
+                //    associated with a given connection handle are in asynchronous mode or all are in synchronous mode.
+                //    A statement handle on a connection cannot be in asynchronous mode while another statement handle
+                //    on the same connection is in synchronous mode, and vice versa.
+                // SQL_AM_STATEMENT = Statement level asynchronous execution is supported.Some statement handles
+                //    associated with a connection handle can be in asynchronous mode, while other statement handles on
+                //    the same connection are in synchronous mode.
+                // SQL_AM_NONE = Asynchronous mode is not supported.
+                intParams[SQL_ASYNC_MODE] = SQL_AM_NONE;
+#endif // SQL_ASYNC_MODE
+
 #ifdef SQL_ASYNC_NOTIFICATION
-                // Indicates if the driver supports asynchronous notification.
-                // SQL_ASYNC_NOTIFICATION_CAPABLE  = Asynchronous execution
-                // notification is supported by the driver.
-                // SQL_ASYNC_NOTIFICATION_NOT_CAPABLE Asynchronous execution
-                // notification is not supported by the driver.
+                // Indicates if the driver supports asynchronous notification:
+                // SQL_ASYNC_NOTIFICATION_CAPABLE Asynchronous execution notification is supported by the driver.
+                // SQL_ASYNC_NOTIFICATION_NOT_CAPABLE Asynchronous execution notification is not supported by the
+                //     driver.
+                //
+                // There are two categories of ODBC asynchronous operations: connection level asynchronous operations
+                // and statement level asynchronous operations. If a driver returns SQL_ASYNC_NOTIFICATION_CAPABLE, it
+                // must support notification for all APIs that it can execute asynchronously.
                 intParams[SQL_ASYNC_NOTIFICATION] = SQL_ASYNC_NOTIFICATION_NOT_CAPABLE;
 #endif // SQL_ASYNC_NOTIFICATION
 
+#ifdef SQL_BATCH_ROW_COUNT
+                // Enumerates the behavior of the driver with respect to the availability of row counts. The following
+                // bitmasks are used together with the information type:
+                // SQL_BRC_ROLLED_UP = Row counts for consecutive INSERT, DELETE, or UPDATE statements are rolled up
+                //     into one. If this bit is not set, row counts are available for each statement.
+                // SQL_BRC_PROCEDURES = Row counts, if any, are available when a batch is executed in a stored
+                //     procedure. If row counts are available, they can be rolled up or individually available,
+                //     depending on the SQL_BRC_ROLLED_UP bit.
+                // SQL_BRC_EXPLICIT = Row counts, if any, are available when a batch is executed directly by calling
+                //     SQLExecute or SQLExecDirect. If row counts are available, they can be rolled up or individually
+                //     available, depending on the SQL_BRC_ROLLED_UP bit.
+                intParams[SQL_BATCH_ROW_COUNT] = SQL_BRC_ROLLED_UP | SQL_BRC_EXPLICIT;
+#endif // SQL_BATCH_ROW_COUNT
+
+#ifdef SQL_BATCH_SUPPORT
+                // Bitmask enumerating the driver's support for batches. The following bitmasks are used to determine
+                // which level is supported:
+                // SQL_BS_SELECT_EXPLICIT = The driver supports explicit batches that can have result - set generating
+                //     statements.
+                // SQL_BS_ROW_COUNT_EXPLICIT = The driver supports explicit batches that can have row - count generating
+                //     statements.
+                // SQL_BS_SELECT_PROC = The driver supports explicit procedures that can have result - set generating
+                //     statements.
+                // SQL_BS_ROW_COUNT_PROC = The driver supports explicit procedures that can have row - count generating
+                //     statements.
+                intParams[SQL_BATCH_SUPPORT] = SQL_BS_ROW_COUNT_EXPLICIT;
+#endif // SQL_BATCH_SUPPORT
+
+#ifdef SQL_BOOKMARK_PERSISTENCE
+                // Bitmask enumerating the operations through which bookmarks persist. The following bitmasks are used
+                // together with the flag to determine through which options bookmarks persist:
+                // SQL_BP_CLOSE = Bookmarks are valid after an application calls SQLFreeStmt with the SQL_CLOSE option,
+                //     or SQLCloseCursor to close the cursor associated with a statement.
+                // SQL_BP_DELETE = The bookmark for a row is valid after that row has been deleted.
+                // SQL_BP_DROP = Bookmarks are valid after an application calls SQLFreeHandle with a HandleType of
+                //     SQL_HANDLE_STMT to drop a statement.
+                // SQL_BP_TRANSACTION = Bookmarks are valid after an application commits or rolls back a transaction.
+                // SQL_BP_UPDATE = The bookmark for a row is valid after any column in that row has been updated,
+                //     including key columns.
+                // SQL_BP_OTHER_HSTMT = A bookmark associated with one statement can be used with another statement.
+                //     Unless SQL_BP_CLOSE or SQL_BP_DROP is specified, the cursor on the first statement must be open.
+                intParams[SQL_BOOKMARK_PERSISTENCE] = 0;
+#endif // SQL_BOOKMARK_PERSISTENCE
+
+#ifdef SQL_CATALOG_LOCATION
+                // Value that indicates the position of the catalog in a qualified table name: SQL_CL_START, SQL_CL_END
+                //
+                // An SQL - 92 Full level-conformant driver will always return SQL_CL_START.A value of 0 is returned if
+                // catalogs are not supported by the data source. This InfoType has been renamed for ODBC 3.0 from the
+                // ODBC 2.0 InfoType SQL_QUALIFIER_LOCATION.
+                intParams[SQL_CATALOG_LOCATION] = 0;
+#endif // SQL_CATALOG_LOCATION
+
+#ifdef SQL_QUALIFIER_LOCATION
+                intParams[SQL_QUALIFIER_LOCATION] = 0;
+#endif // SQL_QUALIFIER_LOCATION
+
 #ifdef SQL_GETDATA_EXTENSIONS
                 // Bitmask enumerating extensions to SQLGetData.
                 intParams[SQL_GETDATA_EXTENSIONS] = SQL_GD_ANY_COLUMN | SQL_GD_ANY_ORDER | SQL_GD_BOUND;
@@ -244,92 +906,100 @@ namespace ignite
 
 #ifdef SQL_CATALOG_USAGE
                 // Bitmask enumerating the statements in which catalogs can be used.
+                // The following bitmasks are used to determine where catalogs can be used:
+                // SQL_CU_DML_STATEMENTS = Catalogs are supported in all Data Manipulation Language statements :
+                //     SELECT, INSERT, UPDATE, DELETE, and if supported, SELECT FOR UPDATE and positioned update and
+                //     delete statements.
+                // SQL_CU_PROCEDURE_INVOCATION = Catalogs are supported in the ODBC procedure invocation statement.
+                // SQL_CU_TABLE_DEFINITION = Catalogs are supported in all table definition statements : CREATE TABLE,
+                //     CREATE VIEW, ALTER TABLE, DROP TABLE, and DROP VIEW.
+                // SQL_CU_INDEX_DEFINITION = Catalogs are supported in all index definition statements : CREATE INDEX
+                //     and DROP INDEX.
+                // SQL_CU_PRIVILEGE_DEFINITION = Catalogs are supported in all privilege definition statements : GRANT
+                //     and REVOKE.
+                //
+                // A value of 0 is returned if catalogs are not supported by the data source.To determine whether
+                // catalogs are supported, an application calls SQLGetInfo with the SQL_CATALOG_NAME information type.
+                // An SQL - 92 Full level-conformant driver will always return a bitmask with all of these bits set.
+                // This InfoType has been renamed for ODBC 3.0 from the ODBC 2.0 InfoType SQL_QUALIFIER_USAGE.
                 intParams[SQL_CATALOG_USAGE] = 0;
 #endif // SQL_CATALOG_USAGE
 
+#ifdef SQL_QUALIFIER_USAGE
+                intParams[SQL_QUALIFIER_USAGE] = 0;
+#endif // SQL_QUALIFIER_USAGE
+
 #ifdef SQL_SCHEMA_USAGE
                 // Bitmask enumerating the statements in which schemas can be used.
-                intParams[SQL_SCHEMA_USAGE] = SQL_SU_DML_STATEMENTS |
-                    SQL_SU_TABLE_DEFINITION | SQL_SU_PRIVILEGE_DEFINITION;
+                intParams[SQL_SCHEMA_USAGE] = SQL_SU_DML_STATEMENTS | SQL_SU_TABLE_DEFINITION |
+                    SQL_SU_PRIVILEGE_DEFINITION | SQL_SU_INDEX_DEFINITION;
 #endif // SQL_SCHEMA_USAGE
 
-#ifdef SQL_MAX_IDENTIFIER_LEN
-                // Indicates the maximum size in characters that the data source
-                // supports for user-defined names.
-                intParams[SQL_MAX_IDENTIFIER_LEN] = 128;
-#endif // SQL_MAX_IDENTIFIER_LEN
-
 #ifdef SQL_AGGREGATE_FUNCTIONS
                 // Bitmask enumerating support for aggregation functions.
-                intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_AVG | SQL_AF_COUNT |
-                    SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN | SQL_AF_SUM;
+                intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_AVG | SQL_AF_COUNT | SQL_AF_MAX | SQL_AF_MIN | SQL_AF_SUM |
+                    SQL_AF_DISTINCT;
 #endif // SQL_AGGREGATE_FUNCTIONS
 
 #ifdef SQL_NUMERIC_FUNCTIONS
-                // Bitmask enumerating the scalar numeric functions supported by
-                // the driver and associated data source.
-                intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN |
-                    SQL_FN_NUM_ATAN | SQL_FN_NUM_ATAN2 | SQL_FN_NUM_CEILING | SQL_FN_NUM_COS | SQL_FN_NUM_COT |
-                    SQL_FN_NUM_EXP | SQL_FN_NUM_FLOOR | SQL_FN_NUM_LOG | SQL_FN_NUM_MOD | SQL_FN_NUM_SIGN |
-                    SQL_FN_NUM_SIN | SQL_FN_NUM_SQRT | SQL_FN_NUM_TAN | SQL_FN_NUM_PI | SQL_FN_NUM_RAND |
-                    SQL_FN_NUM_DEGREES | SQL_FN_NUM_LOG10 | SQL_FN_NUM_POWER | SQL_FN_NUM_RADIANS | SQL_FN_NUM_ROUND |
-                    SQL_FN_NUM_TRUNCATE;
+                // Bitmask enumerating the scalar numeric functions supported by the driver and associated data source.
+                intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN | SQL_FN_NUM_EXP |
+                    SQL_FN_NUM_ATAN | SQL_FN_NUM_ATAN2 | SQL_FN_NUM_CEILING | SQL_FN_NUM_COS | SQL_FN_NUM_TRUNCATE |
+                    SQL_FN_NUM_FLOOR | SQL_FN_NUM_DEGREES | SQL_FN_NUM_POWER | SQL_FN_NUM_RADIANS | SQL_FN_NUM_SIGN |
+                    SQL_FN_NUM_SIN | SQL_FN_NUM_LOG | SQL_FN_NUM_TAN | SQL_FN_NUM_PI | SQL_FN_NUM_MOD | SQL_FN_NUM_COT |
+                    SQL_FN_NUM_LOG10 | SQL_FN_NUM_ROUND | SQL_FN_NUM_SQRT | SQL_FN_NUM_RAND;
 #endif // SQL_NUMERIC_FUNCTIONS
 
 #ifdef SQL_STRING_FUNCTIONS
-                // Bitmask enumerating the scalar string functions supported by the
-                // driver and associated data source.
-                intParams[SQL_STRING_FUNCTIONS] = SQL_FN_STR_ASCII | SQL_FN_STR_BIT_LENGTH | SQL_FN_STR_CHAR |
-                    SQL_FN_STR_CONCAT | SQL_FN_STR_DIFFERENCE | SQL_FN_STR_INSERT | SQL_FN_STR_LEFT |
-                    SQL_FN_STR_LENGTH | SQL_FN_STR_LOCATE | SQL_FN_STR_LTRIM | SQL_FN_STR_OCTET_LENGTH |
+                // Bitmask enumerating the scalar string functions supported by the driver and associated data source.
+                intParams[SQL_STRING_FUNCTIONS] = SQL_FN_STR_ASCII | SQL_FN_STR_BIT_LENGTH | SQL_FN_STR_CHAR_LENGTH |
+                    SQL_FN_STR_CHAR | SQL_FN_STR_CONCAT | SQL_FN_STR_DIFFERENCE | SQL_FN_STR_INSERT | SQL_FN_STR_LEFT |
+                    SQL_FN_STR_LENGTH | SQL_FN_STR_CHARACTER_LENGTH | SQL_FN_STR_LTRIM | SQL_FN_STR_OCTET_LENGTH |
                     SQL_FN_STR_POSITION | SQL_FN_STR_REPEAT | SQL_FN_STR_REPLACE | SQL_FN_STR_RIGHT | SQL_FN_STR_RTRIM |
                     SQL_FN_STR_SOUNDEX | SQL_FN_STR_SPACE | SQL_FN_STR_SUBSTRING | SQL_FN_STR_LCASE | SQL_FN_STR_UCASE |
-                    SQL_FN_STR_LOCATE_2 | SQL_FN_STR_CHAR_LENGTH | SQL_FN_STR_CHARACTER_LENGTH;
+                    SQL_FN_STR_LOCATE_2 | SQL_FN_STR_LOCATE;
 #endif // SQL_STRING_FUNCTIONS
 
 #ifdef SQL_TIMEDATE_FUNCTIONS
-                // Bitmask enumerating the scalar date and time functions supported
-                // by the driver and associated data source.
-                intParams[SQL_TIMEDATE_FUNCTIONS] = SQL_FN_TD_CURRENT_DATE | SQL_FN_TD_CURRENT_TIME |
-                    SQL_FN_TD_CURRENT_TIMESTAMP | SQL_FN_TD_CURDATE | SQL_FN_TD_CURTIME | SQL_FN_TD_DAYNAME |
-                    SQL_FN_TD_DAYOFMONTH | SQL_FN_TD_DAYOFWEEK | SQL_FN_TD_DAYOFYEAR | SQL_FN_TD_EXTRACT |
-                    SQL_FN_TD_HOUR | SQL_FN_TD_MINUTE | SQL_FN_TD_MONTH | SQL_FN_TD_MONTHNAME | SQL_FN_TD_NOW |
-                    SQL_FN_TD_QUARTER | SQL_FN_TD_SECOND | SQL_FN_TD_WEEK | SQL_FN_TD_YEAR;
+                // Bitmask enumerating the scalar date and time functions supported by the driver and associated data
+                // source.
+                intParams[SQL_TIMEDATE_FUNCTIONS] = SQL_FN_TD_CURRENT_DATE | SQL_FN_TD_CURRENT_TIME | SQL_FN_TD_WEEK |
+                    SQL_FN_TD_QUARTER | SQL_FN_TD_SECOND | SQL_FN_TD_CURDATE | SQL_FN_TD_CURTIME | SQL_FN_TD_DAYNAME |
+                    SQL_FN_TD_MINUTE | SQL_FN_TD_DAYOFWEEK | SQL_FN_TD_DAYOFYEAR | SQL_FN_TD_EXTRACT | SQL_FN_TD_HOUR |
+                    SQL_FN_TD_DAYOFMONTH | SQL_FN_TD_MONTH | SQL_FN_TD_MONTHNAME | SQL_FN_TD_NOW | SQL_FN_TD_YEAR |
+                    SQL_FN_TD_CURRENT_TIMESTAMP;
 #endif // SQL_TIMEDATE_FUNCTIONS
 
 #ifdef SQL_TIMEDATE_ADD_INTERVALS
-                // Bitmask enumerating timestamp intervals supported by the driver
-                // and associated data source for the TIMESTAMPADD scalar function.
+                // Bitmask enumerating timestamp intervals supported by the driver and associated data source for the
+                // TIMESTAMPADD scalar function.
                 intParams[SQL_TIMEDATE_ADD_INTERVALS] = 0;
 #endif // SQL_TIMEDATE_ADD_INTERVALS
 
 #ifdef SQL_TIMEDATE_DIFF_INTERVALS
-                // Bitmask enumerating timestamp intervals supported by the driver
-                // and associated data source for the TIMESTAMPDIFF scalar function.
+                // Bitmask enumerating timestamp intervals supported by the driver and associated data source for the
+                // TIMESTAMPDIFF scalar function.
                 intParams[SQL_TIMEDATE_DIFF_INTERVALS] = 0;
 #endif // SQL_TIMEDATE_DIFF_INTERVALS
 
 #ifdef SQL_DATETIME_LITERALS
-                // Bitmask enumerating the SQL-92 datetime literals supported by
-                // the data source.
+                // Bitmask enumerating the SQL-92 datetime literals supported by the data source.
                 intParams[SQL_DATETIME_LITERALS] =  SQL_DL_SQL92_DATE | SQL_DL_SQL92_TIME | SQL_DL_SQL92_TIMESTAMP;
 #endif // SQL_DATETIME_LITERALS
 
 #ifdef SQL_SYSTEM_FUNCTIONS
-                // Bitmask enumerating the scalar system functions supported by the
-                // driver and associated data source.
+                // Bitmask enumerating the scalar system functions supported by the driver and associated data source.
                 intParams[SQL_SYSTEM_FUNCTIONS] = SQL_FN_SYS_USERNAME | SQL_FN_SYS_DBNAME | SQL_FN_SYS_IFNULL;
 #endif // SQL_SYSTEM_FUNCTIONS
 
 #ifdef SQL_CONVERT_FUNCTIONS
-                // Bitmask enumerating the scalar conversion functions supported
-                // by the driver and associated data source.
+                // Bitmask enumerating the scalar conversion functions supported by the driver and associated data
+                // source.
                 intParams[SQL_CONVERT_FUNCTIONS] = SQL_FN_CVT_CONVERT | SQL_FN_CVT_CAST;
 #endif // SQL_CONVERT_FUNCTIONS
 
 #ifdef SQL_OJ_CAPABILITIES
-                // Bitmask enumerating the types of outer joins supported by the
-                // driver and data source.
+                // Bitmask enumerating the types of outer joins supported by the driver and data source.
                 intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_NOT_ORDERED | SQL_OJ_ALL_COMPARISON_OPS;
 #endif // SQL_OJ_CAPABILITIES
 
@@ -339,14 +1009,23 @@ namespace ignite
 #endif // SQL_POS_OPERATIONS
 
 #ifdef SQL_SQL92_NUMERIC_VALUE_FUNCTIONS
-                // Bitmask enumerating the numeric value scalar functions.
-                intParams[SQL_SQL92_NUMERIC_VALUE_FUNCTIONS] = 0;
+                // Bitmask enumerating the numeric value scalar functions that are supported by the driver and the
+                // associated data source, as defined in SQL-92.
+                // The following bitmasks are used to determine which numeric functions are supported :
+                // SQL_SNVF_BIT_LENGTH
+                // SQL_SNVF_CHAR_LENGTH
+                // SQL_SNVF_CHARACTER_LENGTH
+                // SQL_SNVF_EXTRACT
+                // SQL_SNVF_OCTET_LENGTH
+                // SQL_SNVF_POSITION
+                intParams[SQL_SQL92_NUMERIC_VALUE_FUNCTIONS] = SQL_SNVF_BIT_LENGTH | SQL_SNVF_CHARACTER_LENGTH |
+                    SQL_SNVF_EXTRACT | SQL_SNVF_OCTET_LENGTH | SQL_SNVF_POSITION;
 #endif // SQL_SQL92_NUMERIC_VALUE_FUNCTIONS
 
 #ifdef SQL_SQL92_STRING_FUNCTIONS
                 // Bitmask enumerating the string scalar functions.
-                intParams[SQL_SQL92_STRING_FUNCTIONS] = SQL_SSF_LOWER | SQL_SSF_UPPER | SQL_SSF_SUBSTRING |
-                    SQL_SSF_TRIM_BOTH | SQL_SSF_TRIM_LEADING | SQL_SSF_TRIM_TRAILING;
+                intParams[SQL_SQL92_STRING_FUNCTIONS] = SQL_SSF_LOWER | SQL_SSF_UPPER | SQL_SSF_TRIM_TRAILING |
+                    SQL_SSF_SUBSTRING | SQL_SSF_TRIM_BOTH | SQL_SSF_TRIM_LEADING;
 #endif // SQL_SQL92_STRING_FUNCTIONS
 
 #ifdef SQL_SQL92_DATETIME_FUNCTIONS
@@ -356,8 +1035,7 @@ namespace ignite
 #endif // SQL_SQL92_DATETIME_FUNCTIONS
 
 #ifdef SQL_SQL92_VALUE_EXPRESSIONS
-                // Bitmask enumerating the value expressions supported,
-                // as defined in SQL-92.
+                // Bitmask enumerating the value expressions supported, as defined in SQL-92.
                 intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE |
                     SQL_SVE_CAST | SQL_SVE_COALESCE | SQL_SVE_NULLIF;
 #endif // SQL_SQL92_VALUE_EXPRESSIONS
@@ -371,222 +1049,216 @@ namespace ignite
 #endif // SQL_SQL92_PREDICATES
 
 #ifdef SQL_SQL92_RELATIONAL_JOIN_OPERATORS
-                // Bitmask enumerating the relational join operators supported
-                // in a SELECT statement, as defined in SQL-92.
+                // Bitmask enumerating the relational join operators supported in a SELECT statement, as defined
+                // in SQL-92.
                 intParams[SQL_SQL92_RELATIONAL_JOIN_OPERATORS] = SQL_SRJO_CORRESPONDING_CLAUSE | SQL_SRJO_CROSS_JOIN |
                     SQL_SRJO_EXCEPT_JOIN | SQL_SRJO_INNER_JOIN | SQL_SRJO_LEFT_OUTER_JOIN| SQL_SRJO_RIGHT_OUTER_JOIN |
                     SQL_SRJO_NATURAL_JOIN | SQL_SRJO_INTERSECT_JOIN | SQL_SRJO_UNION_JOIN;
 #endif // SQL_SQL92_RELATIONAL_JOIN_OPERATORS
 
 #ifdef SQL_STATIC_CURSOR_ATTRIBUTES1
-                // Bitmask that describes the attributes of a static cursor that
-                // are supported by the driver. This bitmask contains the first
-                // subset of attributes; for the second subset, see
+                // Bitmask that describes the attributes of a static cursor that are supported by the driver. This
+                // bitmask contains the first subset of attributes; for the second subset, see
                 // SQL_STATIC_CURSOR_ATTRIBUTES2.
                 intParams[SQL_STATIC_CURSOR_ATTRIBUTES1] = SQL_CA1_NEXT;
-#endif //SQL_STATIC_CURSOR_ATTRIBUTES1
+#endif // SQL_STATIC_CURSOR_ATTRIBUTES1
 
 #ifdef SQL_STATIC_CURSOR_ATTRIBUTES2
-                // Bitmask that describes the attributes of a static cursor that
-                // are supported by the driver. This bitmask contains the second
-                // subset of attributes; for the first subset, see
+                // Bitmask that describes the attributes of a static cursor that are supported by the driver. This
+                // bitmask contains the second subset of attributes; for the first subset, see
                 // SQL_STATIC_CURSOR_ATTRIBUTES1.
                 intParams[SQL_STATIC_CURSOR_ATTRIBUTES2] = 0;
-#endif //SQL_STATIC_CURSOR_ATTRIBUTES2
+#endif // SQL_STATIC_CURSOR_ATTRIBUTES2
 
 #ifdef SQL_CONVERT_BIGINT
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type BIGINT
                 intParams[SQL_CONVERT_BIGINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
-                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_BIT |
-                    SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT |
-                    SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_BIGINT
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_TIMESTAMP | SQL_CVT_TINYINT |
+                    SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_BIT;
+#endif // SQL_CONVERT_BIGINT
 
 #ifdef SQL_CONVERT_BINARY
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type BINARY
-                intParams[SQL_CONVERT_BINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL |
-                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
-                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
-                    SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_BINARY
+                intParams[SQL_CONVERT_BINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_BIT |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_FLOAT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_DATE |
+                    SQL_CVT_TINYINT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_DECIMAL | SQL_CVT_TIME | SQL_CVT_GUID |
+                    SQL_CVT_TIMESTAMP | SQL_CVT_VARBINARY;
+#endif // SQL_CONVERT_BINARY
 
 #ifdef SQL_CONVERT_BIT
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type BIT
-                intParams[SQL_CONVERT_BIT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC |
-                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT;
-#endif //SQL_CONVERT_BIT
+                intParams[SQL_CONVERT_BIT] = SQL_CVT_BIGINT | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_BIT | SQL_CVT_CHAR | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_TINYINT;
+#endif // SQL_CONVERT_BIT
 
 #ifdef SQL_CONVERT_CHAR
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type CHAR
-                intParams[SQL_CONVERT_CHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_CHAR
+                intParams[SQL_CONVERT_CHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_VARBINARY |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL |
+                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_TIMESTAMP | SQL_CVT_DATE | SQL_CVT_TIME |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_GUID;
+#endif // SQL_CONVERT_CHAR
 
 #ifdef SQL_CONVERT_VARCHAR
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type VARCHAR
-                intParams[SQL_CONVERT_VARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_VARCHAR
+                intParams[SQL_CONVERT_VARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_TINYINT |
+                    SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_BIT |
+                    SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_GUID | SQL_CVT_DATE | SQL_CVT_TIME |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_VARCHAR
 
 #ifdef SQL_CONVERT_LONGVARCHAR
-                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type LONGVARCHAR
-                intParams[SQL_CONVERT_LONGVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_LONGVARCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type
+                // LONGVARCHAR
+                intParams[SQL_CONVERT_LONGVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_DATE | SQL_CVT_TIME |
+                    SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC |
+                    SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_REAL | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_GUID |
+                    SQL_CVT_BIGINT | SQL_CVT_LONGVARBINARY | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_TINYINT | SQL_CVT_FLOAT | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_LONGVARCHAR
 
 #ifdef SQL_CONVERT_WCHAR
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type WCHAR
-                intParams[SQL_CONVERT_WCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_WCHAR
+                intParams[SQL_CONVERT_WCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_VARBINARY |
+                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP |
+                    SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_GUID;
+#endif // SQL_CONVERT_WCHAR
 
 #ifdef SQL_CONVERT_WVARCHAR
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type WVARCHAR
-                intParams[SQL_CONVERT_WVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_WVARCHAR
+                intParams[SQL_CONVERT_WVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_REAL |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_DATE | SQL_CVT_TIME |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_GUID |
+                    SQL_CVT_VARBINARY | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_WVARCHAR
 
 #ifdef SQL_CONVERT_WLONGVARCHAR
-                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type WLONGVARCHAR
-                intParams[SQL_CONVERT_WLONGVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_WLONGVARCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type
+                // WLONGVARCHAR
+                intParams[SQL_CONVERT_WLONGVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_VARBINARY |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_DATE | SQL_CVT_FLOAT |
+                    SQL_CVT_INTEGER | SQL_CVT_SMALLINT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif // SQL_CONVERT_WLONGVARCHAR
+
+#ifdef SQL_CONVERT_GUID
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type GUID
+                intParams[SQL_CONVERT_GUID] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_GUID;
+#endif // SQL_CONVERT_GUID
 
 #ifdef SQL_CONVERT_DATE
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type DATE
-                intParams[SQL_CONVERT_DATE] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
-                    SQL_CVT_DATE | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_DATE
+                intParams[SQL_CONVERT_DATE] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_INTEGER | SQL_CVT_BIGINT |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_DATE
 
 #ifdef SQL_CONVERT_DECIMAL
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type DECIMAL
-                intParams[SQL_CONVERT_DECIMAL] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT |
-                    SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_DECIMAL
+                intParams[SQL_CONVERT_DECIMAL] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_TIMESTAMP |
+                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY;
+#endif // SQL_CONVERT_DECIMAL
 
 #ifdef SQL_CONVERT_DOUBLE
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type DOUBLE
-                intParams[SQL_CONVERT_DOUBLE] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_DOUBLE
+                intParams[SQL_CONVERT_DOUBLE] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_TIMESTAMP |
+                    SQL_CVT_TINYINT | SQL_CVT_BIGINT | SQL_CVT_INTEGER | SQL_CVT_FLOAT | SQL_CVT_REAL | SQL_CVT_DOUBLE |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_SMALLINT | SQL_CVT_LONGVARBINARY;
+#endif // SQL_CONVERT_DOUBLE
 
 #ifdef SQL_CONVERT_FLOAT
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type FLOAT
-                intParams[SQL_CONVERT_FLOAT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
-                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_FLOAT
+                intParams[SQL_CONVERT_FLOAT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY |
+                    SQL_CVT_VARBINARY | SQL_CVT_WCHAR | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP | SQL_CVT_BIT;
+#endif // SQL_CONVERT_FLOAT
 
 #ifdef SQL_CONVERT_REAL
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type REAL
-                intParams[SQL_CONVERT_REAL] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
-                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
-                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_REAL
+                intParams[SQL_CONVERT_REAL] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_FLOAT | SQL_CVT_SMALLINT | SQL_CVT_REAL |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_DOUBLE | SQL_CVT_TINYINT | SQL_CVT_WLONGVARCHAR |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP | SQL_CVT_WCHAR;
+#endif // SQL_CONVERT_REAL
 
 #ifdef SQL_CONVERT_INTEGER
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type INTEGER
-                intParams[SQL_CONVERT_INTEGER] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
-                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY |
-                    SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_INTEGER
+                intParams[SQL_CONVERT_INTEGER] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_TINYINT |
+                    SQL_CVT_SMALLINT | SQL_CVT_BIT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT |
+                    SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_INTEGER
 
 #ifdef SQL_CONVERT_NUMERIC
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type NUMERIC
-                intParams[SQL_CONVERT_NUMERIC] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT |
-                    SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
-                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_NUMERIC
+                intParams[SQL_CONVERT_NUMERIC] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_FLOAT |
+                    SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_NUMERIC
 
 #ifdef SQL_CONVERT_SMALLINT
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type SMALLINT
-                intParams[SQL_CONVERT_SMALLINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT |
-                    SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
-                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
-                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_SMALLINT
+                intParams[SQL_CONVERT_SMALLINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_VARBINARY |
+                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_SMALLINT
 
 #ifdef SQL_CONVERT_TINYINT
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type TINYINT
-                intParams[SQL_CONVERT_TINYINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
-                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
-                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_TINYINT
+                intParams[SQL_CONVERT_TINYINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_TINYINT |
+                    SQL_CVT_SMALLINT | SQL_CVT_BIT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT |
+                    SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif // SQL_CONVERT_TINYINT
 
 #ifdef SQL_CONVERT_TIME
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type TIME
-                intParams[SQL_CONVERT_TIME] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
+                intParams[SQL_CONVERT_TIME] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
                     SQL_CVT_TIME | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_TIME
+#endif // SQL_CONVERT_TIME
 
 #ifdef SQL_CONVERT_TIMESTAMP
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type TIMESTAMP
-                intParams[SQL_CONVERT_TIMESTAMP] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_BINARY |
-                    SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP;
-#endif //SQL_CONVERT_TIMESTAMP
+                intParams[SQL_CONVERT_TIMESTAMP] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_DATE |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_DECIMAL | SQL_CVT_INTEGER | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_TIMESTAMP |
+                    SQL_CVT_BIGINT | SQL_CVT_TIME;
+#endif // SQL_CONVERT_TIMESTAMP
 
 #ifdef SQL_CONVERT_VARBINARY
                 // Bitmask indicates the conversions supported by the CONVERT scalar function for target type VARBINARY
-                intParams[SQL_CONVERT_VARBINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
-                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
-                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE |
+                intParams[SQL_CONVERT_VARBINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_BIT |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL |
+                    SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_DATE | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT |
+                    SQL_CVT_DOUBLE | SQL_CVT_INTEGER | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
                     SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_VARBINARY
+#endif // SQL_CONVERT_VARBINARY
 
 #ifdef SQL_CONVERT_LONGVARBINARY
-                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type LONGVARBINARY
-                intParams[SQL_CONVERT_LONGVARBINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
-                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
-                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE |
-                    SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
-#endif //SQL_CONVERT_LONGVARBINARY
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type
+                // LONGVARBINARY
+                intParams[SQL_CONVERT_LONGVARBINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_BIT | SQL_CVT_TINYINT |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_NUMERIC |
+                    SQL_CVT_DECIMAL | SQL_CVT_FLOAT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_DATE |
+                    SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP |
+                    SQL_CVT_SMALLINT | SQL_CVT_TIME | SQL_CVT_GUID;
+#endif // SQL_CONVERT_LONGVARBINARY
 
 #ifdef SQL_PARAM_ARRAY_ROW_COUNTS
                 // Enumerating the driver's properties regarding the availability of row counts in a parameterized
@@ -602,7 +1274,7 @@ namespace ignite
                 //     conceptually equivalent to treating the statement together with the complete parameter array as
                 //     one atomic unit. Errors are handled the same as if one statement were executed.
                 intParams[SQL_PARAM_ARRAY_ROW_COUNTS] = SQL_PARC_NO_BATCH;
-#endif //SQL_PARAM_ARRAY_ROW_COUNTS
+#endif // SQL_PARAM_ARRAY_ROW_COUNTS
 
 #ifdef SQL_PARAM_ARRAY_SELECTS
                 // Enumerating the driver's properties regarding the availability of result sets in a parameterized
@@ -620,14 +1292,7 @@ namespace ignite
                 // SQL_PAS_NO_SELECT = A driver does not allow a result - set generating statement to be executed with
                 //     an array of parameters.
                 intParams[SQL_PARAM_ARRAY_SELECTS] = SQL_PAS_NO_SELECT;
-#endif //SQL_PARAM_ARRAY_SELECTS
-
-#ifdef SQL_CONVERT_GUID
-                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type GUID
-                intParams[SQL_CONVERT_GUID] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
-                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
-                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_GUID;
-#endif //SQL_CONVERT_GUID
+#endif // SQL_PARAM_ARRAY_SELECTS
 
 #ifdef SQL_SCROLL_OPTIONS
                 // Bitmask enumerating the scroll options supported for scrollable cursors
@@ -640,24 +1305,877 @@ namespace ignite
                 //     than the rowset size.The cursor is keyset - driven inside the keyset and dynamic outside the
                 //     keyset. (ODBC 1.0)
                 intParams[SQL_SCROLL_OPTIONS] = SQL_SO_FORWARD_ONLY | SQL_SO_STATIC;
-#endif //SQL_SCROLL_OPTIONS
+#endif // SQL_SCROLL_OPTIONS
+
+#ifdef SQL_ALTER_DOMAIN
+                // Bitmask enumerating the clauses in the ALTER DOMAIN statement, as defined in SQL-92, supported by the
+                // data source. An SQL-92 Full level-compliant driver will always return all the bitmasks. A return
+                // value of "0" means that the ALTER DOMAIN statement is not supported.
+                //
+                // The SQL - 92 or FIPS conformance level at which this feature must be supported is shown in
+                // parentheses next to each bitmask.
+                //
+                // The following bitmasks are used to determine which clauses are supported :
+                // SQL_AD_ADD_DOMAIN_CONSTRAINT = Adding a domain constraint is supported (Full level).
+                // SQL_AD_ADD_DOMAIN_DEFAULT = <alter domain> <set domain default clause> is supported (Full level).
+                // SQL_AD_CONSTRAINT_NAME_DEFINITION = <constraint name definition clause> is supported for naming
+                //     domain constraint (Intermediate level).
+                // SQL_AD_DROP_DOMAIN_CONSTRAINT = <drop domain constraint clause> is supported (Full level).
+                // SQL_AD_DROP_DOMAIN_DEFAULT = <alter domain> <drop domain default clause> is supported (Full level).
+                //
+                // The following bits specify the supported <constraint attributes> if <add domain constraint> is
+                // supported (the SQL_AD_ADD_DOMAIN_CONSTRAINT bit is set) :
+                // SQL_AD_ADD_CONSTRAINT_DEFERRABLE (Full level)
+                // SQL_AD_ADD_CONSTRAINT_NON_DEFERRABLE (Full level)
+                // SQL_AD_ADD_CONSTRAINT_INITIALLY_DEFERRED (Full level)
+                // SQL_AD_ADD_CONSTRAINT_INITIALLY_IMMEDIATE (Full level)
+                intParams[SQL_ALTER_DOMAIN] = 0;
+#endif // SQL_ALTER_DOMAIN
+
+#ifdef SQL_ALTER_TABLE
+                // Bitmask enumerating the clauses in the ALTER TABLE statement supported by the data source.
+                //
+                // The SQL - 92 or FIPS conformance level at which this feature must be supported is shown in
+                // parentheses next to each bitmask. The following bitmasks are used to determine which clauses are
+                // supported :
+                // SQL_AT_ADD_COLUMN_COLLATION = <add column> clause is supported, with facility to specify column
+                //     collation (Full level) (ODBC 3.0)
+                // SQL_AT_ADD_COLUMN_DEFAULT = <add colu

<TRUNCATED>

[23/50] [abbrv] ignite git commit: IGNITE-6316: SQL: fixed CacheConfiguration persistence logic for DDL operations. This closes #2701.

Posted by yz...@apache.org.
IGNITE-6316: SQL: fixed CacheConfiguration persistence logic for DDL operations. This closes #2701.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 21d7fed8a43ba4606eaf850438fc39fcc3223d1c
Parents: 111d8ab
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Sep 21 18:51:18 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Thu Sep 21 18:56:04 2017 +0300

----------------------------------------------------------------------
 .../pagemem/store/IgnitePageStoreManager.java   |   4 +-
 .../cache/CacheAffinitySharedManager.java       |   4 +-
 .../persistence/file/FilePageStoreManager.java  |   7 +-
 .../processors/query/GridQueryProcessor.java    |  30 ++-
 .../pagemem/NoOpPageStoreManager.java           |   2 +-
 .../IgnitePersistentStoreSchemaLoadTest.java    | 237 ++++++++++++-------
 6 files changed, 184 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/21d7fed8/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index eaa85ad..64c5927 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.pagemem.store;
 import java.nio.ByteBuffer;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
@@ -185,9 +184,10 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
 
     /**
      * @param cacheData Cache configuration.
+     * @param overwrite Whether stored configuration should be overwritten if it exists.
      * @throws IgniteCheckedException If failed.
      */
-    public void storeCacheData(StoredCacheData cacheData) throws IgniteCheckedException;
+    public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException;
     /**
      * @param grpId Cache group ID.
      * @return {@code True} if index store for given cache group existed before node started.

http://git-wip-us.apache.org/repos/asf/ignite/blob/21d7fed8/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 ffb55e4..f446932 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
@@ -2595,8 +2595,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         if (cctx.pageStore() != null && cctx.database().persistenceEnabled() && !cctx.kernalContext().clientNode()) {
             try {
                 cctx.pageStore().storeCacheData(
-                    new StoredCacheData(cfg)
-                );
+                    new StoredCacheData(cfg),
+                    false);
             }
             catch (IgniteCheckedException e) {
                 U.error(log(), "Error while saving cache configuration on disk, cfg = " + cfg, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/21d7fed8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index d60151a..b3eb74a 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -207,9 +207,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void storeCacheData(
-        StoredCacheData cacheData
-    ) throws IgniteCheckedException {
+    @Override public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException {
         File cacheWorkDir = cacheWorkDirectory(cacheData.config());
         File file;
 
@@ -222,10 +220,11 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         else
             file = new File(cacheWorkDir, CACHE_DATA_FILENAME);
 
-        if (!file.exists() || file.length() == 0) {
+        if (overwrite || !file.exists() || file.length() == 0) {
             try {
                 file.createNewFile();
 
+                // Pre-existing file will be truncated upon stream open.
                 try (OutputStream stream = new BufferedOutputStream(new FileOutputStream(file))) {
                     marshaller.marshal(cacheData, stream);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/21d7fed8/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 e791101..51d3411 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
@@ -64,7 +64,9 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 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.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
 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;
@@ -503,8 +505,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (!msg.hasError()) {
                 DynamicCacheDescriptor cacheDesc = ctx.cache().cacheDescriptor(msg.operation().cacheName());
 
-                if (cacheDesc != null && F.eq(cacheDesc.deploymentId(), proposeMsg.deploymentId()))
+                if (cacheDesc != null && F.eq(cacheDesc.deploymentId(), proposeMsg.deploymentId())) {
                     cacheDesc.schemaChangeFinish(msg);
+
+                    saveCacheConfiguration(cacheDesc);
+                }
             }
 
             // Propose message will be used from exchange thread to
@@ -2504,6 +2509,29 @@ private IgniteInternalFuture<Object> rebuildIndexesFromHash(@Nullable final Stri
                     ", sndNodeId=" + msg.senderNodeId() + ']');
         }
     }
+    /**
+     * @param desc cache descriptor.
+     */
+    private void saveCacheConfiguration(DynamicCacheDescriptor desc) {
+        GridCacheSharedContext cctx = ctx.cache().context();
+
+        if (cctx.pageStore() != null && cctx.database().persistenceEnabled() && !cctx.kernalContext().clientNode()) {
+            CacheConfiguration cfg = desc.cacheConfiguration();
+
+            try {
+                StoredCacheData data = new StoredCacheData(cfg);
+
+                if (desc.schema() != null)
+                    data.queryEntities(desc.schema().entities());
+
+                cctx.pageStore().storeCacheData(data, true);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Error while saving cache configuration on disk, cfg = " + cfg, e);
+            }
+        }
+    }
+
 
     /**
      * Unwind pending messages for particular operation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/21d7fed8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
index 8fc2bdb..40887e8c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
@@ -172,7 +172,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void storeCacheData(StoredCacheData cacheData) throws IgniteCheckedException {
+    @Override public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21d7fed8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
index 349f062..b4c08b2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
@@ -18,30 +18,27 @@
 package org.apache.ignite.internal.processors.database;
 
 import java.io.Serializable;
-import java.util.Arrays;
-import java.util.LinkedHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.QueryIndex;
-import org.apache.ignite.cache.QueryIndexType;
+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.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.processors.query.QuerySchema;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 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 static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
@@ -53,21 +50,17 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
     /** */
     private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
-    /** Index name. */
-    private static final String IDX_NAME = "my_idx";
-
     /** Cache name. */
     private static final String TMPL_NAME = "test_cache*";
 
     /** Table name. */
     private static final String TBL_NAME = Person.class.getSimpleName();
 
-    /** Schema name. */
-    private static final String SCHEMA_NAME = "PUBLIC";
-
-    /** Cache name. */
-    private static final String CACHE_NAME = TBL_NAME;
+    /** Name of the cache created with {@code CREATE TABLE}. */
+    private static final String SQL_CACHE_NAME = QueryUtils.createTableCacheName(QueryUtils.DFLT_SCHEMA, TBL_NAME);
 
+    /** Name of the cache created upon cluster start. */
+    private static final String STATIC_CACHE_NAME = TBL_NAME;
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -83,9 +76,31 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
 
         cfg.setPersistentStoreConfiguration(pCfg);
 
+        cfg.setActiveOnStart(true);
+
         return cfg;
     }
 
+    /**
+     * Create node configuration with a cache pre-configured.
+     * @param gridName Node name.
+     * @return Node configuration with a cache pre-configured.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("unchecked")
+    private IgniteConfiguration getConfigurationWithStaticCache(String gridName) throws Exception {
+        IgniteConfiguration cfg = getConfiguration(gridName);
+
+        CacheConfiguration ccfg = cacheCfg(STATIC_CACHE_NAME);
+
+        ccfg.setIndexedTypes(Integer.class, Person.class);
+        ccfg.setSqlEscapeAll(true);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return optimize(cfg);
+    }
+
     /** */
     private CacheConfiguration cacheCfg(String name) {
         CacheConfiguration<?, ?> cfg = new CacheConfiguration<>();
@@ -99,20 +114,6 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
         return cfg;
     }
 
-    /** */
-    private QueryEntity getEntity() {
-        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
-
-        fields.put("id", Integer.class.getName());
-        fields.put("name", String.class.getName());
-
-        QueryEntity entity = new QueryEntity(Integer.class.getName(), Person.class.getName());
-        entity.setFields(fields);
-        entity.setTableName(TBL_NAME);
-
-        return entity;
-    }
-
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, "true");
@@ -132,98 +133,112 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
     }
 
     /** */
-    public void testPersistIndex() throws Exception {
-        IgniteEx ig0 = startGrid(0);
-        startGrid(1);
+    public void testDynamicSchemaChangesPersistence() throws Exception {
+        checkSchemaStateAfterNodeRestart(false);
+    }
 
-        final AtomicInteger cnt = new AtomicInteger();
+    /** */
+    public void testDynamicSchemaChangesPersistenceWithAliveCluster() throws Exception {
+        checkSchemaStateAfterNodeRestart(true);
+    }
 
-        GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ig0.context().cache().context().database();
+    /** */
+    @SuppressWarnings("unchecked")
+    public void testDynamicSchemaChangesPersistenceWithStaticCache() throws Exception {
+        IgniteEx node = startGrid(getConfigurationWithStaticCache(getTestIgniteInstanceName(0)));
 
-        db.addCheckpointListener(new DbCheckpointListener() {
-            @Override public void onCheckpointBegin(Context context) {
-                cnt.incrementAndGet();
-            }
-        });
+        node.active(true);
 
-        QueryIndex idx = new QueryIndex("name");
+        IgniteCache cache = node.cache(STATIC_CACHE_NAME);
 
-        idx.setName(IDX_NAME);
+        assertNotNull(cache);
 
-        ig0.context().query().dynamicTableCreate(SCHEMA_NAME, getEntity(), TMPL_NAME, null, null, null,
-            null, 1, true);
+        CountDownLatch cnt = checkpointLatch(node);
 
-        assert indexCnt(ig0, CACHE_NAME) == 0;
+        assertEquals(0, indexCnt(node, STATIC_CACHE_NAME));
 
-        ig0.context().query().dynamicIndexCreate(CACHE_NAME, SCHEMA_NAME, TBL_NAME, idx, false).get();
+        makeDynamicSchemaChanges(node, STATIC_CACHE_NAME);
 
-        assert indexCnt(ig0, CACHE_NAME) == 1;
+        checkDynamicSchemaChanges(node, STATIC_CACHE_NAME);
 
-        waitForCheckpoint(cnt);
+        cnt.await();
 
-        stopGrid(1);
+        stopGrid(0);
 
-        IgniteEx ig1 = startGrid(1);
+        // Restarting with no-cache configuration - otherwise stored configurations
+        // will be ignored due to cache names duplication.
+        node = startGrid(0);
 
-        assert indexCnt(ig1, CACHE_NAME) == 1;
-    }
+        node.active(true);
 
-    /** */
-    public void testPersistCompositeIndex() throws Exception {
-        IgniteEx ig0 = startGrid(0);
-        startGrid(1);
+        checkDynamicSchemaChanges(node, STATIC_CACHE_NAME);
+    }
 
-        final AtomicInteger cnt = new AtomicInteger();
+    /**
+     * Perform test with cache created with {@code CREATE TABLE}.
+     * @param aliveCluster Whether there should remain an alive node when tested node is restarted.
+     * @throws Exception if failed.
+     */
+    private void checkSchemaStateAfterNodeRestart(boolean aliveCluster) throws Exception {
+        IgniteEx node = startGrid(0);
 
-        GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ig0.context().cache().context().database();
+        node.active(true);
 
-        db.addCheckpointListener(new DbCheckpointListener() {
-            @Override public void onCheckpointBegin(Context context) {
-                cnt.incrementAndGet();
-            }
-        });
+        if (aliveCluster)
+            startGrid(1);
 
-        ig0.context().query().dynamicTableCreate(SCHEMA_NAME, getEntity(), TMPL_NAME, null, null, null, null, 1, true);
+        CountDownLatch cnt = checkpointLatch(node);
 
-        assert indexCnt(ig0, CACHE_NAME) == 0;
+        node.context().query().querySqlFieldsNoCache(
+            new SqlFieldsQuery("create table \"Person\" (\"id\" int primary key, \"name\" varchar)"), false).getAll();
 
-        QueryIndex idx = new QueryIndex(Arrays.asList("id", "name"), QueryIndexType.SORTED);
+        assertEquals(0, indexCnt(node, SQL_CACHE_NAME));
 
-        idx.setName(IDX_NAME);
+        makeDynamicSchemaChanges(node, QueryUtils.DFLT_SCHEMA);
 
-        ig0.context().query().dynamicIndexCreate(CACHE_NAME, SCHEMA_NAME, TBL_NAME, idx, false).get();
+        checkDynamicSchemaChanges(node, SQL_CACHE_NAME);
 
-        assert indexCnt(ig0, CACHE_NAME) == 1;
+        cnt.await();
 
-        waitForCheckpoint(cnt);
+        stopGrid(0);
 
-        stopGrid(1);
+        node = startGrid(0);
 
-        IgniteEx ig1 = startGrid(1);
+        node.active(true);
 
-        assert indexCnt(ig1, CACHE_NAME) == 1;
+        checkDynamicSchemaChanges(node, SQL_CACHE_NAME);
     }
 
     /** */
-    private void waitForCheckpoint(final AtomicInteger cnt) throws IgniteInterruptedCheckedException {
-        final int i = cnt.get();
+    private int indexCnt(IgniteEx node, String cacheName) {
+        DynamicCacheDescriptor desc = node.context().cache().cacheDescriptor(cacheName);
+
+        int cnt = 0;
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return cnt.get() > i;
+        if (desc != null) {
+            QuerySchema schema = desc.schema();
+            if (schema != null) {
+                for (QueryEntity entity : schema.entities())
+                    cnt += entity.getIndexes().size();
             }
-        }, 2000);
+        }
+        return cnt;
     }
 
     /** */
-    private int indexCnt(IgniteEx node, String cacheName) {
-
+    private int colsCnt(IgniteEx node, String cacheName) {
         DynamicCacheDescriptor desc = node.context().cache().cacheDescriptor(cacheName);
 
         int cnt = 0;
 
-        for (QueryEntity entity : desc.schema().entities())
-            cnt += entity.getIndexes().size();
+        if (desc != null) {
+            QuerySchema schema = desc.schema();
+            if (schema != null) {
+
+                for (QueryEntity entity : schema.entities())
+                    cnt += entity.getFields().size();
+            }
+        }
 
         return cnt;
     }
@@ -236,6 +251,50 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
     }
 
     /**
+     * @param node Node whose checkpoint to wait for.
+     * @return Latch released when checkpoint happens.
+     */
+    private CountDownLatch checkpointLatch(IgniteEx node) {
+        final CountDownLatch cnt = new CountDownLatch(1);
+
+        GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)node.context().cache().context().database();
+
+        db.addCheckpointListener(new DbCheckpointListener() {
+            @Override public void onCheckpointBegin(Context ctx) {
+                cnt.countDown();
+            }
+        });
+
+        return cnt;
+    }
+
+    /**
+     * Create dynamic index and column.
+     * @param node Node.
+     * @param schema Schema name.
+     */
+    private void makeDynamicSchemaChanges(IgniteEx node, String schema) {
+        node.context().query().querySqlFieldsNoCache(
+            new SqlFieldsQuery("create index \"my_idx\" on \"Person\" (\"id\", \"name\")").setSchema(schema), false)
+                .getAll();
+
+        node.context().query().querySqlFieldsNoCache(
+            new SqlFieldsQuery("alter table \"Person\" add column \"age\" int").setSchema(schema), false)
+                .getAll();
+    }
+
+    /**
+     * Check that dynamically created schema objects are in place.
+     * @param node Node.
+     * @param cacheName Cache name.
+     */
+    private void checkDynamicSchemaChanges(IgniteEx node, String cacheName) {
+        assertEquals(1, indexCnt(node, cacheName));
+
+        assertEquals(3, colsCnt(node, cacheName));
+    }
+
+    /**
      *
      */
     protected static class Person implements Serializable {
@@ -269,21 +328,19 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
             if (o == null || getClass() != o.getClass())
                 return false;
 
-            IgnitePersistentStoreSchemaLoadTest.Person person = (IgnitePersistentStoreSchemaLoadTest.Person)o;
+            IgnitePersistentStoreSchemaLoadTest.Person person = (IgnitePersistentStoreSchemaLoadTest.Person) o;
 
-            if (id != person.id)
-                return false;
+            return id == person.id && (name != null ? name.equals(person.name) : person.name == null);
 
-            return name != null ? name.equals(person.name) : person.name == null;
         }
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int result = id;
+            int res = id;
 
-            result = 31 * result + (name != null ? name.hashCode() : 0);
+            res = 31 * res + (name != null ? name.hashCode() : 0);
 
-            return result;
+            return res;
         }
     }
 }


[18/50] [abbrv] ignite git commit: Web Console: minor refactoring. (cherry picked from commit abe4ee8)

Posted by yz...@apache.org.
Web Console: minor refactoring.
(cherry picked from commit abe4ee8)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: f5ff60eb72cf6bf55511b2c73f841eb50af59526
Parents: b6d570f
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Sep 21 21:13:59 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Sep 21 21:28:06 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/modules/agent/AgentManager.service.js       | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f5ff60eb/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index 7c84edc..9a736b6 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -101,7 +101,7 @@ export default class IgniteAgentManager {
 
         this.clusterVersion = '2.1.0';
 
-        if (!$root.IgniteDemoMode) {
+        if (!this.isDemoMode()) {
             this.connectionSbj.subscribe({
                 next: ({cluster}) => {
                     const version = _.get(cluster, 'clusterVersion');
@@ -115,6 +115,10 @@ export default class IgniteAgentManager {
         }
     }
 
+    isDemoMode() {
+        return this.$root.IgniteDemoMode;
+    }
+
     available(sinceVersion) {
         return this.Version.since(this.clusterVersion, sinceVersion);
     }
@@ -141,7 +145,7 @@ export default class IgniteAgentManager {
         self.socket.on('agents:stat', ({clusters, count}) => {
             const conn = self.connectionSbj.getValue();
 
-            conn.update(self.$root.IgniteDemoMode, count, clusters);
+            conn.update(self.isDemoMode(), count, clusters);
 
             self.connectionSbj.next(conn);
         });


[21/50] [abbrv] ignite git commit: IGNITE-6228 Avoid closing page store by thread interruption. Fixes #2715

Posted by yz...@apache.org.
IGNITE-6228 Avoid closing page store by thread interruption. Fixes #2715


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 111d8abbe6ec7710c7f0e7ebe6d43f3ccb904dcb
Parents: be8afd4
Author: Alexei Scherbakov <al...@gmail.com>
Authored: Thu Sep 21 17:40:16 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Sep 21 18:10:22 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   5 +
 .../PersistentStoreConfiguration.java           |   6 +-
 .../GridCacheDatabaseSharedManager.java         |   7 +-
 .../cache/persistence/file/AsyncFileIO.java     | 218 +++++++++++++++++++
 .../persistence/file/AsyncFileIOFactory.java    |  52 +++++
 .../cache/persistence/file/FileIOFactory.java   |  25 ++-
 .../cache/persistence/file/FilePageStore.java   |   5 +-
 .../file/FileVersionCheckingFactory.java        |   2 +-
 .../persistence/file/RandomAccessFileIO.java    |  48 ++--
 .../file/RandomAccessFileIOFactory.java         |  14 +-
 .../wal/AbstractWalRecordsIterator.java         |   2 +-
 .../cache/persistence/wal/FileInput.java        |   7 +
 .../wal/FileWriteAheadLogManager.java           |   8 +-
 .../reader/StandaloneWalRecordsIterator.java    |   4 +-
 .../internal/util/future/GridFutureAdapter.java |  16 ++
 .../resources/META-INF/classnames.properties    |   2 +
 .../file/IgnitePdsThreadInterruptionTest.java   | 205 +++++++++++++++++
 .../db/wal/IgniteWalFlushFailoverTest.java      |  22 +-
 .../db/wal/crc/IgniteDataIntegrityTests.java    |  39 ++--
 .../development/utils/IgniteWalConverter.java   |   1 -
 .../IgnitePdsWithIndexingCoreTestSuite.java     |   2 +
 21 files changed, 606 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/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 30d5339..628b165 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -706,6 +706,11 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_ENABLE_FORCIBLE_NODE_KILL = "IGNITE_ENABLE_FORCIBLE_NODE_KILL";
 
     /**
+     * If this property is set, then Ignite will use Async File IO factory by default.
+     */
+    public static final String IGNITE_USE_ASYNC_FILE_IO_FACTORY = "IGNITE_USE_ASYNC_FILE_IO_FACTORY";
+
+    /**
      * If the property is set {@link org.apache.ignite.internal.pagemem.wal.record.TxRecord} records
      * will be logged to WAL.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index 888bf42..abca5a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -17,6 +17,8 @@
 package org.apache.ignite.configuration;
 
 import java.io.Serializable;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
 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.util.typedef.internal.S;
@@ -137,7 +139,9 @@ public class PersistentStoreConfiguration implements Serializable {
     private boolean alwaysWriteFullPages = DFLT_WAL_ALWAYS_WRITE_FULL_PAGES;
 
     /** Factory to provide I/O interface for files */
-    private FileIOFactory fileIOFactory = new RandomAccessFileIOFactory();
+    private FileIOFactory fileIOFactory =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_USE_ASYNC_FILE_IO_FACTORY, false) ?
+        new AsyncFileIOFactory() : new RandomAccessFileIOFactory();
 
     /**
      * Number of sub-intervals the whole {@link #setRateTimeInterval(long)} will be split into to calculate

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/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 eef667e..277143c 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
@@ -144,6 +144,7 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static java.nio.file.StandardOpenOption.READ;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 
@@ -742,7 +743,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @param partFile Partition file.
      */
     private int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, IgniteCheckedException {
-        try (FileIO fileIO = persistenceCfg.getFileIOFactory().create(partFile.toFile(), "r")) {
+        try (FileIO fileIO = persistenceCfg.getFileIOFactory().create(partFile.toFile())) {
             int minimalHdr = FilePageStore.HEADER_SIZE;
 
             if (fileIO.size() < minimalHdr)
@@ -961,7 +962,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 checkpointLock.readLock().unlock();
 
                 try {
-                    checkpointer.wakeupForCheckpoint(0, "too many dirty pages").cpBeginFut.get();
+                    checkpointer.wakeupForCheckpoint(0, "too many dirty pages").cpBeginFut.getUninterruptibly();
                 }
                 catch (IgniteCheckedException e) {
                     throw new IgniteException("Failed to wait for checkpoint begin.", e);
@@ -1390,7 +1391,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteCheckedException {
         buf.position(0);
 
-        try (FileChannel ch = FileChannel.open(cpMarkerFile.toPath(), StandardOpenOption.READ)) {
+        try (FileChannel ch = FileChannel.open(cpMarkerFile.toPath(), READ)) {
             ch.read(buf);
 
             buf.flip();

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
new file mode 100644
index 0000000..8fad7a5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
@@ -0,0 +1,218 @@
+/*
+ * 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.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.CompletionHandler;
+import java.nio.file.OpenOption;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+
+/**
+ * File I/O implementation based on {@link AsynchronousFileChannel}.
+ */
+public class AsyncFileIO implements FileIO {
+    /**
+     * File channel associated with {@code file}
+     */
+    private final AsynchronousFileChannel ch;
+
+    /**
+     * Channel's position.
+     */
+    private volatile long position;
+
+    /** */
+    private final ThreadLocal<ChannelOpFuture> holder;
+
+    /** */
+    private GridConcurrentHashSet<ChannelOpFuture> asyncFuts = new GridConcurrentHashSet<>();
+
+    /**
+     * Creates I/O implementation for specified {@code file}
+     * @param file Random access file
+     * @param modes Open modes.
+     */
+    public AsyncFileIO(File file, ThreadLocal<ChannelOpFuture> holder, OpenOption... modes) throws IOException {
+        this.ch = AsynchronousFileChannel.open(file.toPath(), modes);
+
+        this.holder = holder;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long position() throws IOException {
+        return position;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void position(long newPosition) throws IOException {
+        this.position = newPosition;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(ByteBuffer destinationBuffer) throws IOException {
+        ChannelOpFuture fut = holder.get();
+        fut.reset();
+
+        ch.read(destinationBuffer, position, this, fut);
+
+        try {
+            return fut.getUninterruptibly();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(ByteBuffer destinationBuffer, long position) throws IOException {
+        ChannelOpFuture fut = holder.get();
+        fut.reset();
+
+        ch.read(destinationBuffer, position, null, fut);
+
+        try {
+            return fut.getUninterruptibly();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+        finally {
+            asyncFuts.remove(fut);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(byte[] buffer, int offset, int length) throws IOException {
+        ChannelOpFuture fut = holder.get();
+        fut.reset();
+
+        ch.read(ByteBuffer.wrap(buffer, offset, length), position, this, fut);
+
+        try {
+            return fut.getUninterruptibly();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int write(ByteBuffer sourceBuffer) throws IOException {
+        ChannelOpFuture fut = holder.get();
+        fut.reset();
+
+        ch.write(sourceBuffer, position, this, fut);
+
+        try {
+            return fut.getUninterruptibly();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int write(ByteBuffer sourceBuffer, long position) throws IOException {
+        ChannelOpFuture fut = holder.get();
+        fut.reset();
+
+        asyncFuts.add(fut);
+
+        ch.write(sourceBuffer, position, null, fut);
+
+        try {
+            return fut.getUninterruptibly();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+        finally {
+            asyncFuts.remove(fut);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(byte[] buffer, int offset, int length) throws IOException {
+        ChannelOpFuture fut = holder.get();
+        fut.reset();
+
+        ch.write(ByteBuffer.wrap(buffer, offset, length), position, this, fut);
+
+        try {
+            fut.getUninterruptibly();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void force() throws IOException {
+        ch.force(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size() throws IOException {
+        return ch.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() throws IOException {
+        ch.truncate(0);
+
+        this.position = 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        for (ChannelOpFuture asyncFut : asyncFuts) {
+            try {
+                asyncFut.getUninterruptibly(); // Ignore interrupts while waiting for channel close.
+            }
+            catch (IgniteCheckedException e) {
+                throw new IOException(e);
+            }
+        }
+
+        ch.close();
+    }
+
+    /** */
+    static class ChannelOpFuture extends GridFutureAdapter<Integer> implements CompletionHandler<Integer, AsyncFileIO>  {
+        /** {@inheritDoc} */
+        @Override public void completed(Integer result, AsyncFileIO attachment) {
+            if (attachment != null) {
+                if (result != -1)
+                    attachment.position += result;
+            }
+
+            // Release waiter and allow next operation to begin.
+            super.onDone(result, null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void failed(Throwable exc, AsyncFileIO attachment) {
+            super.onDone(exc);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
new file mode 100644
index 0000000..0fb3052
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.file.OpenOption;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+
+/**
+ * File I/O factory which uses {@link AsynchronousFileChannel} based implementation of FileIO.
+ */
+public class AsyncFileIOFactory implements FileIOFactory {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public FileIO create(File file) throws IOException {
+        return create(file, CREATE, READ, WRITE);
+    }
+
+    /** */
+    private ThreadLocal<AsyncFileIO.ChannelOpFuture> holder = new ThreadLocal<AsyncFileIO.ChannelOpFuture>() {
+        @Override protected AsyncFileIO.ChannelOpFuture initialValue() {
+            return new AsyncFileIO.ChannelOpFuture();
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+        return new AsyncFileIO(file, holder, modes);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
index 0ffc653..c3a75f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
@@ -20,26 +20,29 @@ package org.apache.ignite.internal.processors.cache.persistence.file;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
+import java.nio.file.OpenOption;
 
+/**
+ * {@link FileIO} factory definition.
+ */
 public interface FileIOFactory extends Serializable {
-
     /**
-     * Creates I/O interface for file with default I/O mode
+     * Creates I/O interface for file with default I/O mode.
      *
-     * @param file File
-     * @return File I/O interface
-     * @throws IOException If I/O interface creation was failed
+     * @param file File.
+     * @return File I/O interface.
+     * @throws IOException If I/O interface creation was failed.
      */
-    FileIO create(File file) throws IOException;
+    public FileIO create(File file) throws IOException;
 
     /**
-     * Creates I/O interface for file with specified mode
+     * Creates I/O interface for file with specified mode.
      *
      * @param file File
-     * @param mode I/O mode in
-     * @return File I/O interface
-     * @throws IOException If I/O interface creation was failed
+     * @param modes Open modes.
+     * @return File I/O interface.
+     * @throws IOException If I/O interface creation was failed.
      */
-    FileIO create(File file, String mode) throws IOException;
+    public FileIO create(File file, OpenOption... modes) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/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 6052a7c..98764a2 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
@@ -35,6 +35,9 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDat
 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;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
 
 /**
@@ -400,7 +403,7 @@ public class FilePageStore implements PageStore {
                     IgniteCheckedException err = null;
 
                     try {
-                        this.fileIO = fileIO = ioFactory.create(cfgFile, "rw");
+                        this.fileIO = fileIO = ioFactory.create(cfgFile, CREATE, READ, WRITE);
 
                         if (cfgFile.length() == 0)
                             allocated.set(initFile());

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
index 53bd802..40870dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
@@ -55,7 +55,7 @@ public class FileVersionCheckingFactory implements FilePageStoreFactory {
         if (!file.exists())
             return createPageStore(type, file, latestVersion());
 
-        try (FileIO fileIO = fileIOFactory.create(file, "r")) {
+        try (FileIO fileIO = fileIOFactory.create(file)) {
             int minHdr = FilePageStore.HEADER_SIZE;
 
             if (fileIO.size() < minHdr)

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
index 73a560a..55849fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
@@ -17,94 +17,88 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.file;
 
+import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.file.OpenOption;
 
 /**
- * File I/O implementation based on {@code java.io.RandomAccessFile}.
+ * File I/O implementation based on {@link FileChannel}.
  */
 public class RandomAccessFileIO implements FileIO {
-
-    /**
-     * Random access file associated with this I/O
-     */
-    private final RandomAccessFile file;
-
     /**
-     * File channel associated with {@code file}
+     * File channel.
      */
-    private final FileChannel channel;
+    private final FileChannel ch;
 
     /**
      * Creates I/O implementation for specified {@code file}
      *
-     * @param file Random access file
+     * @param file File.
+     * @param modes Open modes.
      */
-    public RandomAccessFileIO(RandomAccessFile file) {
-        this.file = file;
-        this.channel = file.getChannel();
+    public RandomAccessFileIO(File file, OpenOption... modes) throws IOException {
+        ch = FileChannel.open(file.toPath(), modes);
     }
 
     /** {@inheritDoc} */
     @Override public long position() throws IOException {
-        return channel.position();
+        return ch.position();
     }
 
     /** {@inheritDoc} */
     @Override public void position(long newPosition) throws IOException {
-        channel.position(newPosition);
+        ch.position(newPosition);
     }
 
     /** {@inheritDoc} */
     @Override public int read(ByteBuffer destinationBuffer) throws IOException {
-        return channel.read(destinationBuffer);
+        return ch.read(destinationBuffer);
     }
 
     /** {@inheritDoc} */
     @Override public int read(ByteBuffer destinationBuffer, long position) throws IOException {
-        return channel.read(destinationBuffer, position);
+        return ch.read(destinationBuffer, position);
     }
 
     /** {@inheritDoc} */
     @Override public int read(byte[] buffer, int offset, int length) throws IOException {
-        return file.read(buffer, offset, length);
+        return ch.read(ByteBuffer.wrap(buffer, offset, length));
     }
 
     /** {@inheritDoc} */
     @Override public int write(ByteBuffer sourceBuffer) throws IOException {
-        return channel.write(sourceBuffer);
+        return ch.write(sourceBuffer);
     }
 
     /** {@inheritDoc} */
     @Override public int write(ByteBuffer sourceBuffer, long position) throws IOException {
-        return channel.write(sourceBuffer, position);
+        return ch.write(sourceBuffer, position);
     }
 
     /** {@inheritDoc} */
     @Override public void write(byte[] buffer, int offset, int length) throws IOException {
-        file.write(buffer, offset, length);
+        ch.write(ByteBuffer.wrap(buffer, offset, length));
     }
 
     /** {@inheritDoc} */
     @Override public void force() throws IOException {
-        channel.force(false);
+        ch.force(false);
     }
 
     /** {@inheritDoc} */
     @Override public long size() throws IOException {
-        return channel.size();
+        return ch.size();
     }
 
     /** {@inheritDoc} */
     @Override public void clear() throws IOException {
-        channel.position(0);
-        file.setLength(0);
+        ch.truncate(0);
     }
 
     /** {@inheritDoc} */
     @Override public void close() throws IOException {
-        file.close();
+        ch.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
index 6b731f2..856ba1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
@@ -19,7 +19,11 @@ package org.apache.ignite.internal.processors.cache.persistence.file;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
+import java.nio.file.OpenOption;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
 
 /**
  * File I/O factory which provides RandomAccessFileIO implementation of FileIO.
@@ -30,13 +34,11 @@ public class RandomAccessFileIOFactory implements FileIOFactory {
 
     /** {@inheritDoc} */
     @Override public FileIO create(File file) throws IOException {
-        return create(file, "rw");
+        return create(file, CREATE, READ, WRITE);
     }
 
     /** {@inheritDoc} */
-    @Override public FileIO create(File file, String mode) throws IOException {
-        RandomAccessFile rf = new RandomAccessFile(file, mode);
-
-        return new RandomAccessFileIO(rf);
+    @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+        return new RandomAccessFileIO(file, modes);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
index 2749d5c..d5a2555 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
@@ -258,7 +258,7 @@ public abstract class AbstractWalRecordsIterator
         @Nullable final FileWALPointer start)
         throws IgniteCheckedException, FileNotFoundException {
         try {
-            FileIO fileIO = ioFactory.create(desc.file, "r");
+            FileIO fileIO = ioFactory.create(desc.file);
 
             try {
                 FileInput in = new FileInput(fileIO, buf);

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
index 74edbfa..3b20fce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
@@ -63,6 +63,13 @@ public final class FileInput implements ByteBufferBackedDataInput {
     }
 
     /**
+     * File I/O.
+     */
+    public FileIO io() {
+        return io;
+    }
+
+    /**
      * Clear buffer.
      */
     private void clearBuffer() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/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 18584a8..87069d9 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
@@ -79,6 +79,10 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+
 /**
  * File WAL manager.
  */
@@ -940,7 +944,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         if (log.isDebugEnabled())
             log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']');
 
-        try (FileIO fileIO = ioFactory.create(file, "rw")) {
+        try (FileIO fileIO = ioFactory.create(file, CREATE, READ, WRITE)) {
             int left = psCfg.getWalSegmentSize();
 
             if (mode == WALMode.DEFAULT) {
@@ -1365,7 +1369,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 Files.move(dstTmpFile.toPath(), dstFile.toPath());
 
                 if (mode == WALMode.DEFAULT) {
-                    try (FileIO f0 = ioFactory.create(dstFile, "rw")) {
+                    try (FileIO f0 = ioFactory.create(dstFile, CREATE, READ, WRITE)) {
                         f0.force();
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index 900aab5..c92d572 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -181,7 +181,7 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
 
     /**
      * This methods checks all provided files to be correct WAL segment.
-     * Header record and its position is checked. WAL position is used to deremine real index.
+     * Header record and its position is checked. WAL position is used to determine real index.
      * File index from file name is ignored.
      *
      * @param allFiles files to scan
@@ -202,7 +202,7 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
             FileWALPointer ptr;
 
             try (
-                FileIO fileIO = ioFactory.create(file, "r");
+                FileIO fileIO = ioFactory.create(file);
                 ByteBufferExpander buf = new ByteBufferExpander(HEADER_RECORD_SIZE, ByteOrder.nativeOrder())
             ) {
                 final DataInput in = new FileInput(fileIO, buf);

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index 323babd..f8c0b14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.concurrent.locks.LockSupport;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
@@ -499,6 +500,21 @@ public class GridFutureAdapter<R> implements IgniteInternalFuture<R> {
     }
 
     /**
+     * Resets future for subsequent reuse.
+     */
+    public void reset() {
+        final Object oldState = state;
+
+        if (oldState == INIT)
+            return;
+
+        if (!isDone(oldState))
+            throw new IgniteException("Illegal state");
+
+        compareAndSetState(oldState, INIT);
+    }
+
+    /**
      * Callback to notify that future is cancelled.
      *
      * @return {@code True} if cancel flag was set by this call.

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 2fb8f4b..ad3846f 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -2090,3 +2090,5 @@ org.apache.ignite.transactions.TransactionRollbackException
 org.apache.ignite.transactions.TransactionState
 org.apache.ignite.transactions.TransactionTimeoutException
 org.apache.ignite.util.AttributeNodeFilter
+org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO
+org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.java
new file mode 100644
index 0000000..aab569a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsThreadInterruptionTest.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.internal.processors.cache.persistence.db.file;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jsr166.ThreadLocalRandom8;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Test what interruptions of writing threads do not affect PDS.
+ */
+public class IgnitePdsThreadInterruptionTest extends GridCommonAbstractTest {
+    /** */
+    private static final int PAGE_SIZE = 1 << 12; // 4096
+
+    /** */
+    public static final int THREADS_CNT = 1;
+
+    /**
+     * Cache name.
+     */
+    private final String cacheName = "cache";
+
+    /** */
+    private volatile boolean stop = false;
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPersistentStoreConfiguration(storeConfiguration());
+
+        cfg.setMemoryConfiguration(memoryConfiguration());
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(cacheName));
+
+        return cfg;
+    }
+
+    /**
+     * @return Store config.
+     */
+    private PersistentStoreConfiguration storeConfiguration() {
+        PersistentStoreConfiguration cfg = new PersistentStoreConfiguration();
+
+        cfg.setWalMode(WALMode.LOG_ONLY);
+
+        cfg.setWalFsyncDelayNanos(0);
+
+        cfg.setFileIOFactory(new AsyncFileIOFactory());
+
+        return cfg;
+    }
+
+    /**
+     * @return Memory config.
+     */
+    private MemoryConfiguration memoryConfiguration() {
+        final MemoryConfiguration memCfg = new MemoryConfiguration();
+
+        MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
+        // memPlcCfg.setPageEvictionMode(RANDOM_LRU); TODO Fix NPE on start.
+        memPlcCfg.setName("dfltMemPlc");
+
+        memCfg.setPageSize(PAGE_SIZE);
+        memCfg.setConcurrencyLevel(1);
+        memCfg.setMemoryPolicies(memPlcCfg);
+        memCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+
+        return memCfg;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        deleteWorkFiles();
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+
+        deleteWorkFiles();
+    }
+
+    /**
+     * Tests interruptions on WAL write.
+     *
+     * @throws Exception
+     */
+    public void testInterruptsOnWALWrite() throws Exception {
+        final Ignite ignite = startGrid();
+
+        ignite.active(true);
+
+        final int valLen = 8192;
+
+        final byte[] payload = new byte[valLen];
+
+        final int maxKey = 100_000;
+
+        Thread[] workers = new Thread[THREADS_CNT];
+
+        final AtomicReference<Throwable> fail = new AtomicReference<>();
+
+        Runnable clo = new Runnable() {
+            @Override public void run() {
+                IgniteCache<Object, Object> cache = ignite.cache(cacheName);
+
+                while (!stop)
+                    cache.put(ThreadLocalRandom8.current().nextInt(maxKey), payload);
+            }
+        };
+
+        for (int i = 0; i < workers.length; i++) {
+            workers[i] = new Thread(clo);
+            workers[i].setName("writer-" + i);
+            workers[i].setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+                @Override public void uncaughtException(Thread t, Throwable e) {
+                    fail.compareAndSet(null, e);
+                }
+            });
+        }
+
+        for (Thread worker : workers)
+            worker.start();
+
+        Thread.sleep(3_000);
+
+        // Interrupts should not affect writes.
+        for (Thread worker : workers)
+            worker.interrupt();
+
+        Thread.sleep(3_000);
+
+        stop = true;
+
+        for (Thread worker : workers)
+            worker.join();
+
+        Throwable t = fail.get();
+
+        assert t == null : t;
+
+        IgniteCache<Object, Object> cache = ignite.cache(cacheName);
+
+        int verifiedKeys = 0;
+
+        // Post check.
+        for (int i = 0; i < maxKey; i++) {
+            byte[] val = (byte[]) cache.get(i);
+
+            if (val != null) {
+                assertEquals("Illegal length", valLen, val.length);
+
+                verifiedKeys++;
+            }
+        }
+
+        log.info("Verified keys: " + verifiedKeys);
+    }
+
+    /**
+     * @throws IgniteCheckedException If fail.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
index cad10ae..048e8bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
@@ -42,12 +42,16 @@ 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;
+import java.nio.file.OpenOption;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
 
 /**
  *
  */
 public class IgniteWalFlushFailoverTest extends GridCommonAbstractTest {
-
     /** */
     private static final String TEST_CACHE = "testCache";
 
@@ -168,22 +172,22 @@ public class IgniteWalFlushFailoverTest extends GridCommonAbstractTest {
     private static class FailingFileIOFactory implements FileIOFactory {
         private static final long serialVersionUID = 0L;
 
+        /** */
         private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory();
 
-        @Override
-        public FileIO create(File file) throws IOException {
-            return create(file, "rw");
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file) throws IOException {
+            return create(file, CREATE, READ, WRITE);
         }
 
-        @Override
-        public FileIO create(File file, String mode) throws IOException {
-            FileIO delegate = delegateFactory.create(file, mode);
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+            FileIO delegate = delegateFactory.create(file, modes);
 
             return new FileIODecorator(delegate) {
                 int writeAttempts = 2;
 
-                @Override
-                public int write(ByteBuffer sourceBuffer) throws IOException {
+                @Override public int write(ByteBuffer sourceBuffer) throws IOException {
                     if (--writeAttempts == 0)
                         throw new RuntimeException("Test exception. Unable to write to file.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/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 e4874d9..3d52507 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
@@ -21,11 +21,11 @@ import junit.framework.TestCase;
 import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.util.concurrent.ThreadLocalRandom;
-import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO;
+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.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
@@ -38,9 +38,6 @@ public class IgniteDataIntegrityTests extends TestCase {
     /** File input. */
     private FileInput fileInput;
 
-    /** Random access file. */
-    private RandomAccessFile randomAccessFile;
-
     /** Buffer expander. */
     private ByteBufferExpander expBuf;
 
@@ -51,13 +48,13 @@ public class IgniteDataIntegrityTests extends TestCase {
         File file = File.createTempFile("integrity", "dat");
         file.deleteOnExit();
 
-        randomAccessFile = new RandomAccessFile(file, "rw");
-
         expBuf = new ByteBufferExpander(1024, ByteOrder.BIG_ENDIAN);
 
+        FileIOFactory factory = new RandomAccessFileIOFactory();
+
         fileInput = new FileInput(
-            new RandomAccessFileIO(randomAccessFile),
-            expBuf
+                factory.create(file),
+                expBuf
         );
 
         ByteBuffer buf = ByteBuffer.allocate(1024);
@@ -71,13 +68,15 @@ public class IgniteDataIntegrityTests extends TestCase {
             buf.putInt(PureJavaCrc32.calcCrc32(buf, 12));
         }
 
-        randomAccessFile.write(buf.array());
-        randomAccessFile.getFD().sync();
+        buf.rewind();
+
+        fileInput.io().write(buf);
+        fileInput.io().force();
     }
 
     /** {@inheritDoc} */
     @Override protected void tearDown() throws Exception {
-        randomAccessFile.close();
+        fileInput.io().close();
         expBuf.close();
     }
 
@@ -177,22 +176,24 @@ public class IgniteDataIntegrityTests extends TestCase {
      */
     private void toggleOneRandomBit(int rangeFrom, int rangeTo) throws IOException {
         int pos = ThreadLocalRandom.current().nextInt(rangeFrom, rangeTo);
-        randomAccessFile.seek(pos);
+        fileInput.io().position(pos);
+
+        byte[] buf = new byte[1];
 
-        byte b = randomAccessFile.readByte();
+        fileInput.io().read(buf, 0, 1);
 
-        b ^=  (1 << 3);
+        buf[0] ^= (1 << 3);
 
-        randomAccessFile.seek(pos);
-        randomAccessFile.writeByte(b);
-        randomAccessFile.getFD().sync();
+        fileInput.io().position(pos);
+        fileInput.io().write(buf, 0, 1);
+        fileInput.io().force();
     }
 
     /**
      *
      */
     private void checkIntegrity() throws Exception {
-        randomAccessFile.seek(0);
+        fileInput.io().position(0);
 
         for (int i = 0; i < 1024 / 16; i++) {
             try(FileInput.Crc32CheckingFileInput in = fileInput.startRead(false)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java
----------------------------------------------------------------------
diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java
index b7c3cb7..f3268d9 100644
--- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java
+++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java
@@ -21,7 +21,6 @@ import java.io.File;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
-import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;

http://git-wip-us.apache.org/repos/asf/ignite/blob/111d8abb/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
index ae8ea18..cfbe2e0 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsMulti
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageEvictionTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCacheIntegrationTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsNoActualWalHistoryTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsThreadInterruptionTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRecoveryTxLogicalRecordsTest;
 
@@ -53,6 +54,7 @@ public class IgnitePdsWithIndexingCoreTestSuite extends TestSuite {
 
         suite.addTestSuite(IgnitePdsBinaryMetadataOnClusterRestartTest.class);
         suite.addTestSuite(IgnitePdsMarshallerMappingRestoreOnNodeStartTest.class);
+        suite.addTestSuite(IgnitePdsThreadInterruptionTest.class);
 
         return suite;
     }


[38/50] [abbrv] ignite git commit: IGNITE-6385 Offheap page eviction doesn't work if data is loaded without data streamer - Fixes #2666.

Posted by yz...@apache.org.
IGNITE-6385 Offheap page eviction doesn't work if data is loaded without data streamer - Fixes #2666.

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

(cherry picked from commit 64b4abe)

(cherry picked from commit 7ca611f)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 7daefd2701c7a566d759c2b7de4912ea3f5d37a8
Parents: 1faa8db
Author: Ivan Rakov <iv...@gmail.com>
Authored: Mon Sep 18 11:35:17 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:28:54 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/freelist/PagesList.java   | 35 ++++++-----
 .../paged/PageEvictionDataStreamerTest.java     | 65 ++++++++++++++++++++
 .../paged/PageEvictionMultinodeTest.java        |  6 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |  2 +
 4 files changed, 91 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
index 6c355f6..8a540a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -154,9 +155,8 @@ public abstract class PagesList extends DataStructure {
      */
     protected final void init(long metaPageId, boolean initNew) throws IgniteCheckedException {
         if (metaPageId != 0L) {
-            if (initNew) {
+            if (initNew)
                 init(metaPageId, PagesListMetaIO.VERSIONS.latest());
-            }
             else {
                 Map<Integer, GridLongList> bucketsData = new HashMap<>();
 
@@ -509,10 +509,14 @@ public abstract class PagesList extends DataStructure {
      */
     private Stripe getPageForPut(int bucket) throws IgniteCheckedException {
         // Striped pool optimization.
-        int stripeIdx; IgniteThread igniteThread = IgniteThread.current();
+        IgniteThread igniteThread = IgniteThread.current();
 
-        if (igniteThread != null && (stripeIdx = igniteThread.stripe()) != -1) {
-            Stripe[] tails = getBucket(bucket);
+        Stripe[] tails = getBucket(bucket);
+
+        if (igniteThread != null && igniteThread.policy() == GridIoPolicy.DATA_STREAMER_POOL) {
+            int stripeIdx = igniteThread.stripe();
+
+            assert stripeIdx != -1 : igniteThread;
 
             while (tails == null || stripeIdx >= tails.length) {
                 addStripe(bucket, true);
@@ -523,8 +527,6 @@ public abstract class PagesList extends DataStructure {
             return tails[stripeIdx];
         }
 
-        Stripe[] tails = getBucket(bucket);
-
         if (tails == null)
             return addStripe(bucket, true);
 
@@ -854,9 +856,8 @@ public abstract class PagesList extends DataStructure {
 
                         assert nextPageAddr != 0L;
 
-                        if (locked == null) {
+                        if (locked == null)
                             locked = new GridLongList(6);
-                        }
 
                         locked.add(nextId);
                         locked.add(nextPage);
@@ -911,9 +912,8 @@ public abstract class PagesList extends DataStructure {
                 updateTail(bucket, pageId, prevId);
 
                 // Release write.
-                for (int i = 0; i < locked.size(); i+=3) {
+                for (int i = 0; i < locked.size(); i += 3)
                     writeUnlock(locked.get(i), locked.get(i + 1), locked.get(i + 2), FALSE, true);
-                }
             }
         }
 
@@ -933,9 +933,13 @@ public abstract class PagesList extends DataStructure {
         int len = tails.length;
 
         // Striped pool optimization.
-        int stripeIdx; IgniteThread igniteThread = IgniteThread.current();
+        IgniteThread igniteThread = IgniteThread.current();
+
+        if (igniteThread != null && igniteThread.policy() == GridIoPolicy.DATA_STREAMER_POOL) {
+            int stripeIdx = igniteThread.stripe();
+
+            assert stripeIdx != -1 : igniteThread;
 
-        if (igniteThread != null && (stripeIdx = igniteThread.stripe()) != -1) {
             if (stripeIdx >= len)
                 return null;
 
@@ -971,8 +975,11 @@ public abstract class PagesList extends DataStructure {
         // Striped pool optimization.
         IgniteThread igniteThread = IgniteThread.current();
 
-        if (igniteThread != null && igniteThread.stripe() != -1)
+        if (igniteThread != null && igniteThread.policy() == GridIoPolicy.DATA_STREAMER_POOL) {
+            assert igniteThread.stripe() != -1 : igniteThread;
+
             return writeLock(pageId, page);
+        }
 
         long pageAddr = tryWriteLock(pageId, page);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java
new file mode 100644
index 0000000..6b6d687
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java
@@ -0,0 +1,65 @@
+/*
+* 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.eviction.paged;
+
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class PageEvictionDataStreamerTest extends PageEvictionMultinodeTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_LRU, super.getConfiguration(gridName));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
+        IgniteCache<Object, Object> cache = clientGrid.getOrCreateCache(cfg);
+
+        try (IgniteDataStreamer<Object, Object> ldr = clientGrid.dataStreamer(cfg.getName())) {
+            ldr.allowOverwrite(true);
+
+            for (int i = 1; i <= ENTRIES; i++) {
+                ThreadLocalRandom r = ThreadLocalRandom.current();
+
+                if (r.nextInt() % 5 == 0)
+                    ldr.addData(i, new TestObject(PAGE_SIZE / 4 - 50 + r.nextInt(5000))); // Fragmented object.
+                else
+                    ldr.addData(i, new TestObject(r.nextInt(PAGE_SIZE / 4 - 50))); // Fits in one page.
+
+                if (i % (ENTRIES / 10) == 0)
+                    System.out.println(">>> Entries put: " + i);
+            }
+        }
+
+        int resultingSize = cache.size(CachePeekMode.PRIMARY);
+
+        System.out.println(">>> Resulting size: " + resultingSize);
+
+        // Eviction started, no OutOfMemory occurred, success.
+        assertTrue(resultingSize < ENTRIES);
+
+        clientGrid.destroyCache(cfg.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
index c2c0775..e4587aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
@@ -42,7 +42,7 @@ public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest
         CacheWriteSynchronizationMode.FULL_SYNC, CacheWriteSynchronizationMode.FULL_ASYNC};
 
     /** Client grid. */
-    private Ignite clientGrid;
+    Ignite clientGrid;
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
@@ -94,7 +94,7 @@ public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest
      * @param cfg Config.
      * @throws Exception If failed.
      */
-    private void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
+    protected void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
         IgniteCache<Object, Object> cache = clientGrid.getOrCreateCache(cfg);
 
         for (int i = 1; i <= ENTRIES; i++) {
@@ -121,7 +121,7 @@ public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest
         System.out.println(">>> Resulting size: " + resultingSize);
 
         // Eviction started, no OutOfMemory occurred, success.
-        assertTrue(resultingSize < ENTRIES);
+        assertTrue(resultingSize < ENTRIES * 10 / 11);
 
         clientGrid.destroyCache(cfg.getName());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
index 9f03c60..7985e2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.eviction.fifo.FifoEvictionPol
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearOnlyNearEvictionPolicySelfTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionDataStreamerTest;
 import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionReadThroughTest;
 import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionTouchOrderTest;
 import org.apache.ignite.internal.processors.cache.eviction.paged.Random2LruNearEnabledPageEvictionMultinodeTest;
@@ -79,6 +80,7 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(Random2LruPageEvictionWithRebalanceTest.class));
         suite.addTest(new TestSuite(PageEvictionTouchOrderTest.class));
         suite.addTest(new TestSuite(PageEvictionReadThroughTest.class));
+        suite.addTest(new TestSuite(PageEvictionDataStreamerTest.class));
 
         return suite;
     }


[41/50] [abbrv] ignite git commit: IGNITE-6101 Try to improve local scans performance

Posted by yz...@apache.org.
IGNITE-6101 Try to improve local scans performance

(cherry picked from commit 8d5e73c)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 2310417165d7fd663ff668359ee9a46c775feebf
Parents: 9f5c9e3
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Thu Sep 21 19:45:22 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:29:25 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     |  26 ++
 .../processors/cache/CacheObjectUtils.java      |  63 ++-
 .../processors/cache/GridCacheAdapter.java      |   6 +-
 .../processors/cache/GridCacheEntryEx.java      |  10 +
 .../processors/cache/GridCacheMapEntry.java     |  27 +-
 .../processors/cache/IgniteCacheProxyImpl.java  |  26 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   4 +-
 .../distributed/near/GridNearCacheEntry.java    |   4 +-
 .../processors/cache/query/CacheQueryEntry.java |  58 +++
 .../query/GridCacheDistributedQueryManager.java |  16 +-
 .../cache/query/GridCacheQueryAdapter.java      |  53 ++-
 .../cache/query/GridCacheQueryManager.java      | 466 +++++++++----------
 .../IgniteCacheObjectProcessorImpl.java         | 164 -------
 .../UserCacheObjectByteArrayImpl.java           |  59 +++
 .../cacheobject/UserCacheObjectImpl.java        |  82 ++++
 .../cacheobject/UserKeyCacheObjectImpl.java     | 101 ++++
 .../service/GridServiceProcessor.java           |   6 +-
 .../resources/META-INF/classnames.properties    |  22 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   6 +
 .../GridCacheQueryTransformerSelfTest.java      |  41 ++
 20 files changed, 744 insertions(+), 496 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index 969f3e1..2e0db93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -59,6 +59,12 @@ import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.binary.builder.BinaryLazyValue;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.processors.cache.CacheObjectByteArrayImpl;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
+import org.apache.ignite.internal.processors.cacheobject.UserCacheObjectByteArrayImpl;
+import org.apache.ignite.internal.processors.cacheobject.UserCacheObjectImpl;
+import org.apache.ignite.internal.processors.cacheobject.UserKeyCacheObjectImpl;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -701,6 +707,26 @@ public class BinaryUtils {
     }
 
     /**
+     * @param obj Object to check.
+     * @return True if this is an object of a known type.
+     */
+    public static boolean knownCacheObject(Object obj) {
+        if (obj == null)
+            return false;
+
+        Class<?> cls= obj.getClass();
+
+        return cls == KeyCacheObjectImpl.class ||
+            cls == BinaryObjectImpl.class ||
+            cls == CacheObjectImpl.class ||
+            cls == CacheObjectByteArrayImpl.class ||
+            cls == BinaryEnumObjectImpl.class ||
+            cls == UserKeyCacheObjectImpl.class ||
+            cls == UserCacheObjectImpl.class ||
+            cls == UserCacheObjectByteArrayImpl.class;
+    }
+
+    /**
      * @param arr Array to check.
      * @return {@code true} if this array is of a known type.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
index f9c76df..e942924 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
@@ -17,12 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.internal.binary.BinaryUtils;
-import org.apache.ignite.internal.util.typedef.F;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.util.typedef.F;
 
 /**
  * Cache object utility methods.
@@ -34,10 +33,35 @@ public class CacheObjectUtils {
      * @param cpy Copy value flag.
      * @return Unwrapped object.
      */
+    public static Object unwrapBinaryIfNeeded(CacheObjectValueContext ctx, CacheObject o, boolean keepBinary, boolean cpy) {
+        return unwrapBinary(ctx, o, keepBinary, cpy);
+    }
+
+    /**
+     * @param o Object to unwrap.
+     * @param keepBinary Keep binary flag.
+     * @param cpy Copy value flag.
+     * @return Unwrapped object.
+     */
     public static Object unwrapBinaryIfNeeded(CacheObjectValueContext ctx, Object o, boolean keepBinary, boolean cpy) {
         if (o == null)
             return null;
 
+        // TODO has to be overloaded
+        if (o instanceof Map.Entry) {
+            Map.Entry entry = (Map.Entry)o;
+
+            Object key = entry.getKey();
+
+            Object uKey = unwrapBinary(ctx, key, keepBinary, cpy);
+
+            Object val = entry.getValue();
+
+            Object uVal = unwrapBinary(ctx, val, keepBinary, cpy);
+
+            return (key != uKey || val != uVal) ? F.t(uKey, uVal) : o;
+        }
+
         return unwrapBinary(ctx, o, keepBinary, cpy);
     }
 
@@ -84,7 +108,10 @@ public class CacheObjectUtils {
         Map<Object, Object> map0 = BinaryUtils.newMap(map);
 
         for (Map.Entry<Object, Object> e : map.entrySet())
-            map0.put(unwrapBinary(ctx, e.getKey(), false, cpy), unwrapBinary(ctx, e.getValue(), false, cpy));
+            // TODO why don't we use keepBinary parameter here?
+            map0.put(
+                unwrapBinary(ctx, e.getKey(), false, cpy),
+                unwrapBinary(ctx, e.getValue(), false, cpy));
 
         return map0;
     }
@@ -103,7 +130,7 @@ public class CacheObjectUtils {
             col0 = new ArrayList<>(col.size());
 
         for (Object obj : col)
-            col0.add(unwrapBinary(ctx, obj, keepBinary, cpy));
+            col0.add(unwrapBinaryIfNeeded(ctx, obj, keepBinary, cpy));
 
         return col0;
     }
@@ -135,31 +162,25 @@ public class CacheObjectUtils {
      */
     @SuppressWarnings("unchecked")
     private static Object unwrapBinary(CacheObjectValueContext ctx, Object o, boolean keepBinary, boolean cpy) {
-        if (o instanceof Map.Entry) {
-            Map.Entry entry = (Map.Entry)o;
-
-            Object key = entry.getKey();
-
-            Object uKey = unwrapBinary(ctx, key, keepBinary, cpy);
+        if (o == null)
+            return o;
 
-            Object val = entry.getValue();
+        while (BinaryUtils.knownCacheObject(o)) {
+            CacheObject co = (CacheObject)o;
 
-            Object uVal = unwrapBinary(ctx, val, keepBinary, cpy);
+            if (!co.isPlatformType() && keepBinary)
+                return o;
 
-            return (key != uKey || val != uVal) ? F.t(uKey, uVal) : o;
+            // It may be a collection of binaries
+            o = co.value(ctx, cpy);
         }
-        else if (BinaryUtils.knownCollection(o))
+
+        if (BinaryUtils.knownCollection(o))
             return unwrapKnownCollection(ctx, (Collection<Object>)o, keepBinary, cpy);
         else if (BinaryUtils.knownMap(o))
             return unwrapBinariesIfNeeded(ctx, (Map<Object, Object>)o, keepBinary, cpy);
         else if (o instanceof Object[])
             return unwrapBinariesInArrayIfNeeded(ctx, (Object[])o, keepBinary, cpy);
-        else if (o instanceof CacheObject) {
-            CacheObject co = (CacheObject)o;
-
-            if (!keepBinary || co.isPlatformType())
-                return unwrapBinary(ctx, co.value(ctx, cpy), keepBinary, cpy);
-        }
 
         return o;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 40f2b49..92a8245 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -83,13 +83,13 @@ import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
@@ -3911,7 +3911,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         return ctx.itHolder().iterator(iter, new CacheIteratorConverter<Cache.Entry<K, V>, Map.Entry<K, V>>() {
             @Override protected Cache.Entry<K, V> convert(Map.Entry<K, V> e) {
-                return new CacheEntryImpl<>(e.getKey(), e.getValue());
+                // Actually Scan Query returns Iterator<CacheQueryEntry> by default,
+                // CacheQueryEntry implements both Map.Entry and Cache.Entry interfaces.
+                return (Cache.Entry<K, V>) e;
             }
 
             @Override protected void remove(Cache.Entry<K, V> item) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index e2bc7ff..b2cabac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -934,6 +935,15 @@ public interface GridCacheEntryEx {
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
+     * @param row Already extracted value.
+     * @return Value.
+     * @throws IgniteCheckedException If failed to read from swap storage.
+     * @throws GridCacheEntryRemovedException If entry was removed.
+     */
+    @Nullable public CacheObject unswap(CacheDataRow row)
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
+
+    /**
      * Unswap ignoring flags.
      *
      * @param needVal If {@code false} then do not need to deserialize value during unswap.

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/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 765dff5..958f156 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
@@ -342,9 +342,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
+    @Override public final CacheObject unswap(CacheDataRow row) throws IgniteCheckedException, GridCacheEntryRemovedException {
+        row = unswap(row, true);
+
+        return row != null ? row.value() : null;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public final CacheObject unswap(boolean needVal)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
-        CacheDataRow row = unswap(needVal, true);
+        CacheDataRow row = unswap(null, true);
 
         return row != null ? row.value() : null;
     }
@@ -352,13 +359,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /**
      * Unswaps an entry.
      *
-     * @param needVal If {@code false} then do not to deserialize value during unswap.
+     * @param row Already extracted cache data.
      * @param checkExpire If {@code true} checks for expiration, as result entry can be obsoleted or marked deleted.
      * @return Value.
      * @throws IgniteCheckedException If failed.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    @Nullable protected CacheDataRow unswap(boolean needVal, boolean checkExpire)
+    @Nullable protected CacheDataRow unswap(@Nullable CacheDataRow row, boolean checkExpire)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         boolean obsolete = false;
         boolean deferred = false;
@@ -368,7 +375,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             checkObsolete();
 
             if (isStartVersion() && ((flags & IS_UNSWAPPED_MASK) == 0)) {
-                CacheDataRow read = cctx.offheap().read(this);
+                assert row == null || row.key() == key: "Unexpected row key";
+
+                CacheDataRow read = row == null ? cctx.offheap().read(this) : row;
 
                 flags |= IS_UNSWAPPED_MASK;
 
@@ -572,7 +581,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (val == null) {
                     if (isStartVersion()) {
-                        unswap(true, false);
+                        unswap(null, false);
 
                         val = this.val;
                     }
@@ -1322,7 +1331,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             // Load and remove from swap if it is new.
             if (isNew())
-                oldRow = unswap(retval, false);
+                oldRow = unswap(null, false);
 
             old = val;
 
@@ -2411,7 +2420,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     return null;
 
                 if (val == null && offheap)
-                    unswap(true, false);
+                    unswap(null, false);
 
                 if (checkExpired()) {
                     if (cctx.deferredDelete()) {
@@ -2648,7 +2657,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         boolean isNew = isStartVersion();
 
         if (isNew)
-            unswap(true, false);
+            unswap(null, false);
 
         CacheObject val = this.val;
 
@@ -2952,7 +2961,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 checkObsolete();
 
                 if (isStartVersion())
-                    unswap(true, false);
+                    unswap(null, false);
 
                 long expireTime = expireTimeExtras();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
index 337c1bb..3056361 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
@@ -375,31 +375,7 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
         final GridCloseableIterator<R> iter = ctx.kernalContext().query().executeQuery(GridCacheQueryType.SCAN,
             ctx.name(), ctx, new IgniteOutClosureX<GridCloseableIterator<R>>() {
                 @Override public GridCloseableIterator<R> applyx() throws IgniteCheckedException {
-                    final GridCloseableIterator iter0 = qry.executeScanQuery();
-
-                    final boolean needToConvert = transformer == null;
-
-                    return new GridCloseableIteratorAdapter<R>() {
-                        @Override protected R onNext() throws IgniteCheckedException {
-                            Object next = iter0.nextX();
-
-                            if (needToConvert) {
-                                Map.Entry<K, V> entry = (Map.Entry<K, V>)next;
-
-                                return (R)new CacheEntryImpl<>(entry.getKey(), entry.getValue());
-                            }
-
-                            return (R)next;
-                        }
-
-                        @Override protected boolean onHasNext() throws IgniteCheckedException {
-                            return iter0.hasNextX();
-                        }
-
-                        @Override protected void onClose() throws IgniteCheckedException {
-                            iter0.close();
-                        }
-                    };
+                    return qry.executeScanQuery();
                 }
             }, true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
index 7da3d4f..5566bb4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
@@ -22,8 +22,8 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
@@ -53,7 +53,7 @@ public class GridDhtDetachedCacheEntry extends GridDistributedCacheEntry {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public CacheDataRow unswap(boolean needVal, boolean checkExpire) throws IgniteCheckedException {
+    @Nullable @Override public CacheDataRow unswap(CacheDataRow row, boolean checkExpire) throws IgniteCheckedException {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index 6e606bf..ce728b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -30,9 +30,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvcc;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.F;
@@ -443,7 +443,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public CacheDataRow unswap(boolean needVal, boolean checkExpire) {
+    @Nullable @Override public CacheDataRow unswap(CacheDataRow row, boolean checkExpire) {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryEntry.java
new file mode 100644
index 0000000..4787464
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryEntry.java
@@ -0,0 +1,58 @@
+/*
+ * 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.query;
+
+import javax.cache.Cache;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.internal.processors.cache.CacheEntryImplEx;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
+
+/** */
+final class CacheQueryEntry<K,V> extends IgniteBiTuple<K,V> implements Cache.Entry<K,V> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    public CacheQueryEntry() {
+        // No-op.
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     */
+    CacheQueryEntry(@Nullable K key, @Nullable V val) {
+        super(key, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> cls) {
+        if (cls != null && cls.isAssignableFrom(getClass()))
+            return cls.cast(this);
+
+        if (cls.isAssignableFrom(CacheEntryImpl.class))
+            return (T)new CacheEntryImpl<>(getKey(), getValue());
+
+        if (cls.isAssignableFrom(CacheEntry.class))
+            return (T)new CacheEntryImplEx<>(getKey(), getValue(), null);
+
+        throw new IllegalArgumentException("Unwrapping to class is not supported: " + cls);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index 7f859a2..b860f02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -42,7 +42,6 @@ import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 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.P1;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -632,7 +631,20 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                 if (locIter != null && locIter.hasNextX())
                     cur = locIter.nextX();
 
-                return cur != null || (cur = fut.next()) != null;
+                return cur != null || (cur = convert(fut.next())) != null;
+            }
+
+            /**
+             * @param obj Entry to convert.
+             * @return Cache entry
+             */
+            private Object convert(Object obj) {
+                if(qry.transform() != null)
+                    return obj;
+
+                Map.Entry e = (Map.Entry)obj;
+
+                return e == null ? null : new CacheQueryEntry(e.getKey(), e.getValue());
             }
 
             @Override protected void onClose() throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 023c03c..c4eae8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Deque;
@@ -517,7 +518,8 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     @Override public GridCloseableIterator executeScanQuery() throws IgniteCheckedException {
         assert type == SCAN : "Wrong processing of qyery: " + type;
 
-        Collection<ClusterNode> nodes = nodes();
+        // Affinity nodes snapshot.
+        Collection<ClusterNode> nodes = new ArrayList<>(nodes());
 
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -537,13 +539,15 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
         final GridCacheQueryManager qryMgr = cctx.queries();
 
-        if (part != null && !cctx.isLocal())
-            return new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx);
-        else {
-            boolean loc = nodes.size() == 1 && F.first(nodes).id().equals(cctx.localNodeId());
+        boolean loc = nodes.size() == 1 && F.first(nodes).id().equals(cctx.localNodeId());
 
-            return loc ? qryMgr.scanQueryLocal(this, true) : qryMgr.scanQueryDistributed(this, nodes);
-        }
+        if (loc)
+            return qryMgr.scanQueryLocal(this, true);
+
+        if (part != null)
+            return new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx);
+        else
+            return qryMgr.scanQueryDistributed(this, nodes);
     }
 
     /**
@@ -621,12 +625,12 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     /**
      * Wrapper for queries with fallback.
      */
-    private static class ScanQueryFallbackClosableIterator extends GridCloseableIteratorAdapter<Map.Entry> {
+    private static class ScanQueryFallbackClosableIterator extends GridCloseableIteratorAdapter {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** Query future. */
-        private volatile T2<GridCloseableIterator<Map.Entry>, GridCacheQueryFutureAdapter> tuple;
+        private volatile T2<GridCloseableIterator<Object>, GridCacheQueryFutureAdapter> tuple;
 
         /** Backups. */
         private volatile Queue<ClusterNode> nodes;
@@ -653,7 +657,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         private boolean firstItemReturned;
 
         /** */
-        private Map.Entry cur;
+        private Object cur;
 
         /**
          * @param part Partition.
@@ -726,7 +730,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                 }
             }
             else {
-                final GridCacheQueryBean bean = new GridCacheQueryBean(qry, null, null, null);
+                final GridCacheQueryBean bean = new GridCacheQueryBean(qry, null, qry.transform, null);
 
                 GridCacheQueryFutureAdapter fut =
                     (GridCacheQueryFutureAdapter)qryMgr.queryDistributed(bean, Collections.singleton(node));
@@ -736,13 +740,13 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         }
 
         /** {@inheritDoc} */
-        @Override protected Map.Entry onNext() throws IgniteCheckedException {
+        @Override protected Object onNext() throws IgniteCheckedException {
             if (!onHasNext())
                 throw new NoSuchElementException();
 
             assert cur != null;
 
-            Map.Entry e = cur;
+            Object e = cur;
 
             cur = null;
 
@@ -755,9 +759,9 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                 if (cur != null)
                     return true;
 
-                T2<GridCloseableIterator<Map.Entry>, GridCacheQueryFutureAdapter> t = tuple;
+                T2<GridCloseableIterator<Object>, GridCacheQueryFutureAdapter> t = tuple;
 
-                GridCloseableIterator<Map.Entry> iter = t.get1();
+                GridCloseableIterator<Object> iter = t.get1();
 
                 if (iter != null) {
                     boolean hasNext = iter.hasNext();
@@ -773,14 +777,14 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                     assert fut != null;
 
                     if (firstItemReturned)
-                        return (cur = (Map.Entry)fut.next()) != null;
+                        return (cur = convert(fut.next())) != null;
 
                     try {
                         fut.awaitFirstPage();
 
                         firstItemReturned = true;
 
-                        return (cur = (Map.Entry)fut.next()) != null;
+                        return (cur = convert(fut.next())) != null;
                     }
                     catch (IgniteClientDisconnectedCheckedException e) {
                         throw CU.convertToCacheException(e);
@@ -793,6 +797,19 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         }
 
         /**
+         * @param obj Entry to convert.
+         * @return Cache entry
+         */
+        private Object convert(Object obj) {
+            if(qry.transform() != null)
+                return obj;
+
+            Map.Entry e = (Map.Entry)obj;
+
+            return e == null ? null : new CacheQueryEntry(e.getKey(), e.getValue());
+        }
+
+        /**
          * @param e Exception for query run.
          */
         private void retryIfPossible(IgniteCheckedException e) {
@@ -847,7 +864,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         @Override protected void onClose() throws IgniteCheckedException {
             super.onClose();
 
-            T2<GridCloseableIterator<Map.Entry>, GridCacheQueryFutureAdapter> t = tuple;
+            T2<GridCloseableIterator<Object>, GridCacheQueryFutureAdapter> t = tuple;
 
             if (t != null && t.get1() != null)
                 t.get1().close();

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 4307d26..05e8a20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -40,10 +40,10 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import javax.cache.Cache;
-import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 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.QueryIndexType;
 import org.apache.ignite.cache.query.QueryMetrics;
@@ -63,7 +63,9 @@ import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheMetricsImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.CacheObjectUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
@@ -71,10 +73,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate;
 import org.apache.ignite.internal.processors.datastructures.SetItemKey;
@@ -817,22 +819,22 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @throws IgniteCheckedException If failed to get iterator.
      */
     @SuppressWarnings({"unchecked"})
-    private GridCloseableIterator<IgniteBiTuple<K, V>> scanIterator(final GridCacheQueryAdapter<?> qry, boolean locNode)
+    private GridCloseableIterator scanIterator(final GridCacheQueryAdapter<?> qry, boolean locNode)
         throws IgniteCheckedException {
         final IgniteBiPredicate<K, V> keyValFilter = qry.scanFilter();
 
         try {
             injectResources(keyValFilter);
 
-            Integer part = qry.partition();
-
-            if (cctx.isLocal())
-                part = null;
+            Integer part = cctx.isLocal() ? null : qry.partition();
 
             if (part != null && (part < 0 || part >= cctx.affinity().partitions()))
-                return new GridEmptyCloseableIterator<>();
-
-            final ExpiryPolicy plc = cctx.expiry();
+                return new GridEmptyCloseableIterator() {
+                    @Override public void close() throws IgniteCheckedException {
+                        closeScanFilter(keyValFilter);
+                        super.close();
+                    }
+                };
 
             AffinityTopologyVersion topVer = GridQueryProcessor.getRequestAffinityTopologyVersion();
 
@@ -854,13 +856,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     throw new GridDhtUnreservedPartitionException(part, cctx.affinity().affinityTopologyVersion(),
                         "Partition can not be reserved");
 
-                if (locPart0.state() != OWNING) {
-                    locPart0.release();
-
-                    throw new GridDhtUnreservedPartitionException(part, cctx.affinity().affinityTopologyVersion(),
-                        "Partition can not be reserved");
-                }
-
                 locPart = locPart0;
 
                 it = cctx.offheap().cachePartitionIterator(cctx.cacheId(), part);
@@ -868,19 +863,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             else {
                 locPart = null;
 
+                // TODO shouldn't we reserve all involved partitions?
                 it = cctx.offheap().cacheIterator(cctx.cacheId(), true, backups, topVer);
             }
 
-            return new PeekValueExpiryAwareIterator(it, plc, topVer, keyValFilter, qry.keepBinary(), locNode) {
-                @Override protected void onClose() {
-                    super.onClose();
-
-                    if (locPart != null)
-                        locPart.release();
-
-                    closeScanFilter(keyValFilter);
-                }
-            };
+            return new ScanQueryIterator(it, qry, topVer, locPart, keyValFilter, locNode, cctx, log);
         }
         catch (IgniteCheckedException | RuntimeException e) {
             closeScanFilter(keyValFilter);
@@ -1185,9 +1172,16 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 final boolean readEvt = cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ);
 
-                while (!Thread.currentThread().isInterrupted() && iter.hasNext()) {
+                CacheObjectContext objCtx = cctx.cacheObjectContext();
+
+                while (!Thread.currentThread().isInterrupted()) {
                     long start = statsEnabled ? System.nanoTime() : 0L;
 
+                    // Need to call it after gathering start time because
+                    // actual row extracting may happen inside this method.
+                    if(!iter.hasNext())
+                        break;
+
                     IgniteBiTuple<K, V> row = iter.next();
 
                     // Query is cancelled.
@@ -1245,8 +1239,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     V val0 = null;
 
                     if (readEvt) {
-                        key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary());
-                        val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary());
+                        key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, qry.keepBinary(), false);
+                        val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, qry.keepBinary(), false);
 
                         switch (type) {
                             case SQL:
@@ -1316,9 +1310,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                     if (rdc != null || trans != null) {
                         if (key0 == null)
-                            key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary());
+                            key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, qry.keepBinary(), false);
                         if (val0 == null)
-                            val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary());
+                            val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, qry.keepBinary(), false);
 
                         Cache.Entry<K, V> entry = new CacheEntryImpl(key0, val0);
 
@@ -1406,22 +1400,24 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * Process local scan query.
      *
      * @param qry Query.
-     * @param updStatisticsIfNeeded Update statistics flag.
+     * @param updateStatistics Update statistics flag.
      */
     @SuppressWarnings({"unchecked", "serial"})
     protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry,
-        final boolean updStatisticsIfNeeded) throws IgniteCheckedException {
+        boolean updateStatistics) throws IgniteCheckedException {
         if (!enterBusy())
             throw new IllegalStateException("Failed to process query request (grid is stopping).");
 
         final boolean statsEnabled = cctx.config().isStatisticsEnabled();
 
-        boolean needUpdStatistics = updStatisticsIfNeeded && statsEnabled;
+        updateStatistics &= statsEnabled;
 
         long startTime = U.currentTimeMillis();
 
         final String namex = cctx.name();
 
+        final IgniteBiPredicate<K, V> scanFilter = qry.scanFilter();
+
         try {
             assert qry.type() == SCAN;
 
@@ -1429,7 +1425,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 log.debug("Running local SCAN query: " + qry);
 
             final String taskName = cctx.kernalContext().task().resolveTaskName(qry.taskHash());
-            final IgniteBiPredicate filter = qry.scanFilter();
             final ClusterNode locNode = cctx.localNode();
             final UUID subjId = qry.subjectId();
 
@@ -1442,80 +1437,23 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     namex,
                     null,
                     null,
-                    filter,
+                    scanFilter,
                     null,
                     null,
                     subjId,
                     taskName));
             }
 
-            final GridCloseableIterator<IgniteBiTuple<K, V>> iter = scanIterator(qry, true);
+            GridCloseableIterator it = scanIterator(qry, true);
 
-            if (updStatisticsIfNeeded)
-                needUpdStatistics = false;
-
-            final boolean readEvt = cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ);
-
-            return new GridCloseableIteratorAdapter<Object>() {
-                @Override protected Object onNext() throws IgniteCheckedException {
-                    long start = statsEnabled ? System.nanoTime() : 0L;
+            updateStatistics = false;
 
-                    IgniteBiTuple<K, V> next = iter.nextX();
-
-                    if (statsEnabled) {
-                        CacheMetricsImpl metrics = cctx.cache().metrics0();
-
-                        metrics.onRead(true);
-
-                        metrics.addGetTimeNanos(System.nanoTime() - start);
-                    }
-
-                    if (readEvt) {
-                        cctx.gridEvents().record(new CacheQueryReadEvent<>(
-                            cctx.localNode(),
-                            "Scan query entry read.",
-                            EVT_CACHE_QUERY_OBJECT_READ,
-                            CacheQueryType.SCAN.name(),
-                            namex,
-                            null,
-                            null,
-                            filter,
-                            null,
-                            null,
-                            subjId,
-                            taskName,
-                            next.getKey(),
-                            next.getValue(),
-                            null,
-                            null));
-                    }
-
-                    IgniteClosure transform = qry.transform();
-
-                    if (transform == null)
-                        return next;
-
-                    Cache.Entry<K, V> entry;
-
-                    if (qry.keepBinary())
-                        entry = cctx.cache().keepBinary().getEntry(next.getKey());
-                    else
-                        entry = cctx.cache().getEntry(next.getKey());
-
-                    return transform.apply(entry);
-                }
-
-                @Override protected boolean onHasNext() throws IgniteCheckedException {
-                    return iter.hasNextX();
-                }
-
-                @Override protected void onClose() throws IgniteCheckedException {
-                    iter.close();
-                }
-            };
+            return it;
         }
         catch (Exception e) {
-            if (needUpdStatistics)
+            closeScanFilter(scanFilter);
+
+            if (updateStatistics)
                 cctx.queries().collectMetrics(GridCacheQueryType.SCAN, namex, startTime,
                     U.currentTimeMillis() - startTime, true);
 
@@ -2031,8 +1969,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         private static final long serialVersionUID = 0L;
 
         /**
-         * Number of fields to report when no fields defined.
-         * Includes _key and _val columns.
+         * Number of fields to report when no fields defined. Includes _key and _val columns.
          */
         private static final int NO_FIELDS_COLUMNS_COUNT = 2;
 
@@ -2846,14 +2783,68 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
-     *
+     * The map prevents put to the map in case the specified request has been removed previously.
      */
-    private class PeekValueExpiryAwareIterator extends GridCloseableIteratorAdapter<IgniteBiTuple<K, V>> {
+    private class RequestFutureMap extends LinkedHashMap<Long, GridFutureAdapter<QueryResult<K, V>>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Count of canceled keys */
+        private static final int CANCELED_COUNT = 128;
+
+        /**
+         * The ID of the canceled request is stored to the set in case remove(reqId) is called before put(reqId,
+         * future).
+         */
+        private Set<Long> canceled;
+
+        /** {@inheritDoc} */
+        @Override public GridFutureAdapter<QueryResult<K, V>> remove(Object key) {
+            if (containsKey(key))
+                return super.remove(key);
+            else {
+                if (canceled == null) {
+                    canceled = Collections.newSetFromMap(
+                        new LinkedHashMap<Long, Boolean>() {
+                            @Override protected boolean removeEldestEntry(Map.Entry<Long, Boolean> eldest) {
+                                return size() > CANCELED_COUNT;
+                            }
+                        });
+                }
+
+                canceled.add((Long)key);
+
+                return null;
+            }
+        }
+
+        /**
+         * @return true if the key is canceled
+         */
+        boolean isCanceled(Long key) {
+            return canceled != null && canceled.contains(key);
+        }
+    }
+
+    /** */
+    private static final class ScanQueryIterator<K, V> extends GridCloseableIteratorAdapter<Object> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        private final ExpiryPolicy plc;
+        private final GridDhtCacheAdapter dht;
+
+        /** */
+        private final GridDhtLocalPartition locPart;
+
+        /** */
+        private final IgniteBiPredicate<K, V> scanFilter;
+
+        /** */
+        private final boolean statsEnabled;
+
+        /** */
+        private final GridIterator<CacheDataRow> it;
 
         /** */
         private final GridCacheAdapter cache;
@@ -2862,73 +2853,94 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         private final AffinityTopologyVersion topVer;
 
         /** */
-        private final GridDhtCacheAdapter dht;
+        private final boolean keepBinary;
 
         /** */
-        private final IgniteBiPredicate<K, V> keyValFilter;
+        private final boolean readEvt;
 
         /** */
-        private boolean locNode;
+        private final String cacheName;
 
         /** */
-        private final boolean keepBinary;
+        private final UUID subjId;
 
         /** */
-        private IgniteBiTuple<K, V> next;
+        private final String taskName;
 
         /** */
-        private IgniteCacheExpiryPolicy expiryPlc;
+        private final IgniteClosure transform;
+
+        /** */
+        private final CacheObjectContext objCtx;
+
+        /** */
+        private final GridCacheContext cctx;
+
+        /** */
+        private final IgniteLogger log;
 
         /** */
-        private GridIterator<CacheDataRow> it;
+        private Object next;
 
-        /** Need advance. */
+        /** */
         private boolean needAdvance;
 
+        /** */
+        private IgniteCacheExpiryPolicy expiryPlc;
+
         /**
          * @param it Iterator.
-         * @param plc Expiry policy.
+         * @param qry Query.
          * @param topVer Topology version.
-         * @param keyValFilter Key-value filter.
-         * @param keepBinary Keep binary flag from the query.
-         * @param locNode Local node.
+         * @param locPart Local partition.
+         * @param scanFilter Scan filter.
+         * @param locNode Local node flag.
+         * @param cctx Cache context.
+         * @param log Logger.
          */
-        private PeekValueExpiryAwareIterator(
+        ScanQueryIterator(
             GridIterator<CacheDataRow> it,
-            ExpiryPolicy plc,
+            GridCacheQueryAdapter qry,
             AffinityTopologyVersion topVer,
-            IgniteBiPredicate<K, V> keyValFilter,
-            boolean keepBinary,
-            boolean locNode
-        ) {
+            GridDhtLocalPartition locPart,
+            IgniteBiPredicate<K, V> scanFilter,
+            boolean locNode,
+            GridCacheContext cctx,
+            IgniteLogger log) {
             this.it = it;
-            this.plc = plc;
             this.topVer = topVer;
-            this.keyValFilter = keyValFilter;
-            this.locNode = locNode;
+            this.locPart = locPart;
+            this.scanFilter = scanFilter;
+            this.cctx = cctx;
+            this.log = log;
 
-            dht = cctx.isLocal() ? null : (cctx.isNear() ? cctx.near().dht() : cctx.dht());
-            cache = dht != null ? dht : cctx.cache();
+            statsEnabled = locNode && cctx.config().isStatisticsEnabled();
 
-            this.keepBinary = keepBinary;
-            expiryPlc = cctx.cache().expiryPolicy(plc);
+            readEvt = locNode && cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ);
 
-            needAdvance = true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean onHasNext() {
-            if (needAdvance) {
-                advance();
-
-                needAdvance = false;
+            if(readEvt){
+                taskName = cctx.kernalContext().task().resolveTaskName(qry.taskHash());
+                subjId = qry.subjectId();
+            }
+            else {
+                taskName = null;
+                subjId = null;
             }
 
-            return next != null;
+            // keep binary for remote scans if possible
+            keepBinary = (!locNode && scanFilter == null) || qry.keepBinary();
+            transform = qry.transform();
+            dht = cctx.isLocal() ? null : (cctx.isNear() ? cctx.near().dht() : cctx.dht());
+            cache = dht != null ? dht : cctx.cache();
+            objCtx = cctx.cacheObjectContext();
+            cacheName = cctx.name();
+
+            needAdvance = true;
+            expiryPlc = this.cctx.cache().expiryPolicy(null);
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteBiTuple<K, V> onNext() {
+        @Override protected Object onNext() {
             if (needAdvance)
                 advance();
             else
@@ -2941,26 +2953,64 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
+        @Override protected boolean onHasNext() {
+            if (needAdvance) {
+                advance();
+
+                needAdvance = false;
+            }
+
+            return next != null;
+        }
+
+        /** {@inheritDoc} */
         @Override protected void onClose() {
-            sendTtlUpdate();
+            if (expiryPlc != null && dht != null) {
+                dht.sendTtlUpdateRequest(expiryPlc);
+
+                expiryPlc = null;
+            }
+
+            if (locPart != null)
+                locPart.release();
+
+            closeScanFilter(scanFilter);
         }
 
         /**
          * Moves the iterator to the next cache entry.
          */
         private void advance() {
-            IgniteBiTuple<K, V> next0 = null;
+            long start = statsEnabled ? System.nanoTime() : 0L;
+
+            Object next = null;
 
             while (it.hasNext()) {
                 CacheDataRow row = it.next();
 
                 KeyCacheObject key = row.key();
-
                 CacheObject val;
 
                 if (expiryPlc != null) {
                     try {
-                        val = value(key);
+                        CacheDataRow tmp = row;
+
+                        while (true) {
+                            try {
+                                GridCacheEntryEx entry = cache.entryEx(key);
+
+                                entry.unswap(tmp);
+
+                                val = entry.peek(true, true, topVer, expiryPlc);
+
+                                cctx.evicts().touch(entry, topVer);
+
+                                break;
+                            }
+                            catch (GridCacheEntryRemovedException ignore) {
+                                tmp = null;
+                            }
+                        }
                     }
                     catch (IgniteCheckedException e) {
                         if (log.isDebugEnabled())
@@ -2969,122 +3019,58 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         val = null;
                     }
 
-                    if (dht != null && expiryPlc.readyToFlush(100)) {
+                    if (dht != null && expiryPlc.readyToFlush(100))
                         dht.sendTtlUpdateRequest(expiryPlc);
-
-                        expiryPlc = cctx.cache().expiryPolicy(plc);
-                    }
                 }
                 else
                     val = row.value();
 
                 if (val != null) {
-                    boolean keepBinary0 = !locNode || keepBinary;
+                    K key0 = (K)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, key, keepBinary, false);
+                    V val0 = (V)CacheObjectUtils.unwrapBinaryIfNeeded(objCtx, val, keepBinary, false);
 
-                    next0 = F.t(
-                        (K)cctx.unwrapBinaryIfNeeded(key, keepBinary0),
-                        (V)cctx.unwrapBinaryIfNeeded(val, keepBinary0));
+                    if (statsEnabled) {
+                        CacheMetricsImpl metrics = cctx.cache().metrics0();
 
-                    boolean passPred = true;
+                        metrics.onRead(true);
 
-                    if (keyValFilter != null) {
-                        Object key0 = next0.getKey();
-                        Object val0 = next0.getValue();
+                        metrics.addGetTimeNanos(System.nanoTime() - start);
+                    }
 
-                        if (keepBinary0 && !keepBinary) {
-                            key0 = (K)cctx.unwrapBinaryIfNeeded(key0, keepBinary);
-                            val0 = (V)cctx.unwrapBinaryIfNeeded(val0, keepBinary);
+                    if (scanFilter == null || scanFilter.apply(key0, val0)) {
+                        if (readEvt && cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ)) {
+                            cctx.gridEvents().record(new CacheQueryReadEvent<>(
+                                cctx.localNode(),
+                                "Scan query entry read.",
+                                EVT_CACHE_QUERY_OBJECT_READ,
+                                CacheQueryType.SCAN.name(),
+                                cacheName,
+                                null,
+                                null,
+                                scanFilter,
+                                null,
+                                null,
+                                subjId,
+                                taskName,
+                                key0,
+                                val0,
+                                null,
+                                null));
                         }
 
-                        passPred = keyValFilter.apply((K)key0, (V)val0);
-                    }
+                        next = transform == null ? new CacheQueryEntry<>(key0, val0)
+                            : transform.apply(new CacheQueryEntry<>(key0, val0));
 
-                    if (passPred)
                         break;
-                    else
-                        next0 = null;
+                    }
                 }
             }
 
-            next = next0;
-
-            if (next == null)
-                sendTtlUpdate();
-        }
-
-        /**
-         * Sends TTL update.
-         */
-        private void sendTtlUpdate() {
-            if (dht != null && expiryPlc != null) {
+            if ((this.next = next) == null && expiryPlc != null && dht != null) {
                 dht.sendTtlUpdateRequest(expiryPlc);
 
                 expiryPlc = null;
             }
         }
-
-        /**
-         * @param key Key.
-         * @return Value.
-         * @throws IgniteCheckedException If failed to peek value.
-         */
-        private CacheObject value(KeyCacheObject key) throws IgniteCheckedException {
-            while (true) {
-                try {
-                    GridCacheEntryEx entry = cache.entryEx(key);
-
-                    entry.unswap();
-
-                    return entry.peek(true, true, topVer, expiryPlc);
-                }
-                catch (GridCacheEntryRemovedException ignore) {
-                    // No-op.
-                }
-            }
-        }
-    }
-
-    /**
-     * The map prevents put to the map in case the specified request has been removed previously.
-     */
-    private class RequestFutureMap extends LinkedHashMap<Long, GridFutureAdapter<QueryResult<K, V>>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Count of canceled keys */
-        private static final int CANCELED_COUNT = 128;
-
-        /**
-         * The ID of the canceled request is stored to the set in case
-         * remove(reqId) is called before put(reqId, future).
-         */
-        private Set<Long> canceled;
-
-        /** {@inheritDoc} */
-        @Override public GridFutureAdapter<QueryResult<K, V>> remove(Object key) {
-            if (containsKey(key))
-                return super.remove(key);
-            else {
-                if (canceled == null) {
-                    canceled = Collections.newSetFromMap(
-                        new LinkedHashMap<Long, Boolean>() {
-                            @Override protected boolean removeEldestEntry(Map.Entry<Long, Boolean> eldest) {
-                                return size() > CANCELED_COUNT;
-                            }
-                        });
-                }
-
-                canceled.add((Long)key);
-
-                return null;
-            }
-        }
-
-        /**
-         * @return true if the key is canceled
-         */
-        boolean isCanceled(Long key) {
-            return canceled != null && canceled.contains(key);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 70711e5..17be90f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cacheobject;
 
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.UUID;
@@ -40,7 +39,6 @@ import org.apache.ignite.internal.processors.cache.IncompleteCacheObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
 import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -341,166 +339,4 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         return false;
     }
 
-    /**
-     * Wraps key provided by user, must be serialized before stored in cache.
-     */
-    private static class UserKeyCacheObjectImpl extends KeyCacheObjectImpl {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         *
-         */
-        public UserKeyCacheObjectImpl() {
-            //No-op.
-        }
-
-        /**
-         * @param key Key.
-         * @param part Partition.
-         */
-        UserKeyCacheObjectImpl(Object key, int part) {
-            super(key, null, part);
-        }
-
-        /**
-         * @param key Key.
-         * @param valBytes Marshalled key.
-         * @param part Partition.
-         */
-        UserKeyCacheObjectImpl(Object key, byte[] valBytes, int part) {
-            super(key, valBytes, part);
-        }
-
-        /** {@inheritDoc} */
-        @Override public KeyCacheObject copy(int part) {
-            if (this.partition() == part)
-                return this;
-
-            return new UserKeyCacheObjectImpl(val, valBytes, part);
-        }
-
-        /** {@inheritDoc} */
-        @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-            try {
-                IgniteCacheObjectProcessor proc = ctx.kernalContext().cacheObjects();
-
-                if (!proc.immutable(val)) {
-                    if (valBytes == null)
-                        valBytes = proc.marshal(ctx, val);
-
-                    boolean p2pEnabled = ctx.kernalContext().config().isPeerClassLoadingEnabled();
-
-                    ClassLoader ldr = p2pEnabled ?
-                        IgniteUtils.detectClassLoader(IgniteUtils.detectClass(this.val)) : U.gridClassLoader();
-
-                    Object val = proc.unmarshal(ctx, valBytes, ldr);
-
-                    KeyCacheObject key = new KeyCacheObjectImpl(val, valBytes, partition());
-
-                    key.partition(partition());
-
-                    return key;
-                }
-
-                KeyCacheObject key = new KeyCacheObjectImpl(val, valBytes, partition());
-
-                key.partition(partition());
-
-                return key;
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException("Failed to marshal object: " + val, e);
-            }
-        }
-    }
-
-    /**
-     * Wraps value provided by user, must be serialized before stored in cache.
-     */
-    private static class UserCacheObjectImpl extends CacheObjectImpl {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         *
-         */
-        public UserCacheObjectImpl() {
-            //No-op.
-        }
-
-        /**
-         * @param val Value.
-         * @param valBytes Value bytes.
-         */
-        public UserCacheObjectImpl(Object val, byte[] valBytes) {
-            super(val, valBytes);
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public <T> T value(CacheObjectValueContext ctx, boolean cpy) {
-            return super.value(ctx, false); // Do not need copy since user value is not in cache.
-        }
-
-        /** {@inheritDoc} */
-        @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-            try {
-                IgniteCacheObjectProcessor proc = ctx.kernalContext().cacheObjects();
-
-                if (valBytes == null)
-                    valBytes = proc.marshal(ctx, val);
-
-                if (ctx.storeValue()) {
-                    boolean p2pEnabled = ctx.kernalContext().config().isPeerClassLoadingEnabled();
-
-                    ClassLoader ldr = p2pEnabled ?
-                        IgniteUtils.detectClass(this.val).getClassLoader() : val.getClass().getClassLoader();
-
-                    Object val = this.val != null && proc.immutable(this.val) ? this.val :
-                        proc.unmarshal(ctx, valBytes, ldr);
-
-                    return new CacheObjectImpl(val, valBytes);
-                }
-
-                return new CacheObjectImpl(null, valBytes);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException("Failed to marshal object: " + val, e);
-            }
-        }
-    }
-
-    /**
-     * Wraps value provided by user, must be copied before stored in cache.
-     */
-    private static class UserCacheObjectByteArrayImpl extends CacheObjectByteArrayImpl {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         *
-         */
-        public UserCacheObjectByteArrayImpl() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        public UserCacheObjectByteArrayImpl(byte[] val) {
-            super(val);
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public <T> T value(CacheObjectValueContext ctx, boolean cpy) {
-            return super.value(ctx, false); // Do not need copy since user value is not in cache.
-        }
-
-        /** {@inheritDoc} */
-        @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-            byte[] valCpy = Arrays.copyOf(val, val.length);
-
-            return new CacheObjectByteArrayImpl(valCpy);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectByteArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectByteArrayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectByteArrayImpl.java
new file mode 100644
index 0000000..aa4d5f5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectByteArrayImpl.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cacheobject;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectByteArrayImpl;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wraps value provided by user, must be copied before stored in cache.
+ */
+public class UserCacheObjectByteArrayImpl extends CacheObjectByteArrayImpl {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public UserCacheObjectByteArrayImpl() {
+        // No-op.
+    }
+
+    /**
+     * @param val Value.
+     */
+    public UserCacheObjectByteArrayImpl(byte[] val) {
+        super(val);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T value(CacheObjectValueContext ctx, boolean cpy) {
+        return super.value(ctx, false); // Do not need copy since user value is not in cache.
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        byte[] valCpy = Arrays.copyOf(val, val.length);
+
+        return new CacheObjectByteArrayImpl(valCpy);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectImpl.java
new file mode 100644
index 0000000..241c12b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserCacheObjectImpl.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cacheobject;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wraps value provided by user, must be serialized before stored in cache.
+ */
+public class UserCacheObjectImpl extends CacheObjectImpl {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public UserCacheObjectImpl() {
+        //No-op.
+    }
+
+    /**
+     * @param val Value.
+     * @param valBytes Value bytes.
+     */
+    public UserCacheObjectImpl(Object val, byte[] valBytes) {
+        super(val, valBytes);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T value(CacheObjectValueContext ctx, boolean cpy) {
+        return super.value(ctx, false); // Do not need copy since user value is not in cache.
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        try {
+            IgniteCacheObjectProcessor proc = ctx.kernalContext().cacheObjects();
+
+            if (valBytes == null)
+                valBytes = proc.marshal(ctx, val);
+
+            if (ctx.storeValue()) {
+                boolean p2pEnabled = ctx.kernalContext().config().isPeerClassLoadingEnabled();
+
+                ClassLoader ldr = p2pEnabled ?
+                    IgniteUtils.detectClass(this.val).getClassLoader() : val.getClass().getClassLoader();
+
+                Object val = this.val != null && proc.immutable(this.val) ? this.val :
+                    proc.unmarshal(ctx, valBytes, ldr);
+
+                return new CacheObjectImpl(val, valBytes);
+            }
+
+            return new CacheObjectImpl(null, valBytes);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to marshal object: " + val, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserKeyCacheObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserKeyCacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserKeyCacheObjectImpl.java
new file mode 100644
index 0000000..de57667
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/UserKeyCacheObjectImpl.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.cacheobject;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Wraps key provided by user, must be serialized before stored in cache.
+ */
+public class UserKeyCacheObjectImpl extends KeyCacheObjectImpl {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public UserKeyCacheObjectImpl() {
+        //No-op.
+    }
+
+    /**
+     * @param key Key.
+     * @param part Partition.
+     */
+    UserKeyCacheObjectImpl(Object key, int part) {
+        super(key, null, part);
+    }
+
+    /**
+     * @param key Key.
+     * @param valBytes Marshalled key.
+     * @param part Partition.
+     */
+    UserKeyCacheObjectImpl(Object key, byte[] valBytes, int part) {
+        super(key, valBytes, part);
+    }
+
+    /** {@inheritDoc} */
+    @Override public KeyCacheObject copy(int part) {
+        if (this.partition() == part)
+            return this;
+
+        return new UserKeyCacheObjectImpl(val, valBytes, part);
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        try {
+            IgniteCacheObjectProcessor proc = ctx.kernalContext().cacheObjects();
+
+            if (!proc.immutable(val)) {
+                if (valBytes == null)
+                    valBytes = proc.marshal(ctx, val);
+
+                boolean p2pEnabled = ctx.kernalContext().config().isPeerClassLoadingEnabled();
+
+                ClassLoader ldr = p2pEnabled ?
+                    IgniteUtils.detectClassLoader(IgniteUtils.detectClass(this.val)) : U.gridClassLoader();
+
+                Object val = proc.unmarshal(ctx, valBytes, ldr);
+
+                KeyCacheObject key = new KeyCacheObjectImpl(val, valBytes, partition());
+
+                key.partition(partition());
+
+                return key;
+            }
+
+            KeyCacheObject key = new KeyCacheObjectImpl(val, valBytes, partition());
+
+            key.partition(partition());
+
+            return key;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to marshal object: " + val, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 46fcfea..1d8720c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -26,7 +26,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
@@ -61,7 +60,6 @@ import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
-import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheIteratorConverter;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -1309,7 +1307,9 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
             return cache.context().itHolder().iterator(iter,
                 new CacheIteratorConverter<Cache.Entry<Object, Object>, Map.Entry<Object, Object>>() {
                     @Override protected Cache.Entry<Object, Object> convert(Map.Entry<Object, Object> e) {
-                        return new CacheEntryImpl<>(e.getKey(), e.getValue());
+                        // Actually Scan Query returns Iterator<CacheQueryEntry> by default,
+                        // CacheQueryEntry implements both Map.Entry and Cache.Entry interfaces.
+                        return (Cache.Entry<Object, Object>)e;
                     }
 
                     @Override protected void remove(Cache.Entry<Object, Object> item) {


[27/50] [abbrv] ignite git commit: IGNITE-6120 Fixed version check for ignite-2.2.0. (cherry picked from commit f4ead8b)

Posted by yz...@apache.org.
IGNITE-6120 Fixed version check for ignite-2.2.0.
(cherry picked from commit f4ead8b)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: ccc8c9b40d09e1d3314a519f1e9417e79c150d9d
Parents: fc34091
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Sep 15 23:19:16 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Sep 22 09:58:11 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/modules/agent/AgentManager.service.js       | 4 +++-
 .../web-console/frontend/app/modules/sql/sql.controller.js   | 8 ++++----
 2 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ccc8c9b4/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index 9a736b6..c24a92f 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -26,6 +26,8 @@ const State = {
     CONNECTED: 'CONNECTED'
 };
 
+const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1'];
+
 class ConnectionState {
     constructor(cluster) {
         this.agents = [];
@@ -499,7 +501,7 @@ export default class IgniteAgentManager {
      */
     querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) {
         if (this.available('2.0.0')) {
-            const task = this.available('2.1.4-p1') ?
+            const task = this.available(...LAZY_QUERY_SINCE) ?
                 this.visorTask('querySqlX2', nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) :
                 this.visorTask('querySqlX2', nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ccc8c9b4/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index c43dc5c..7d8df7a 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -28,9 +28,9 @@ const ROW_IDX = {value: -2, type: 'java.lang.Integer', label: 'ROW_IDX'};
 
 const NON_COLLOCATED_JOINS_SINCE = '1.7.0';
 
-const ENFORCE_JOIN_VERS = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], ['1.9.1']];
+const ENFORCE_JOIN_SINCE = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], '1.9.1'];
 
-const LAZY_QUERY_VERS = ['2.1.4-p1'];
+const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1'];
 
 const _fullColName = (col) => {
     const res = [];
@@ -1367,7 +1367,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const cache = _.find($scope.caches, {name: paragraph.cacheName});
 
             if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...ENFORCE_JOIN_VERS));
+                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...ENFORCE_JOIN_SINCE));
 
             return false;
         };
@@ -1376,7 +1376,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const cache = _.find($scope.caches, {name: paragraph.cacheName});
 
             if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...LAZY_QUERY_VERS));
+                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...LAZY_QUERY_SINCE));
 
             return false;
         };


[40/50] [abbrv] ignite git commit: IGNITE-6101 Try to improve local scans performance

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index ad3846f..e64363d 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -183,6 +183,7 @@ org.apache.ignite.igfs.mapreduce.records.IgfsByteDelimiterRecordResolver
 org.apache.ignite.igfs.mapreduce.records.IgfsFixedLengthRecordResolver
 org.apache.ignite.igfs.mapreduce.records.IgfsNewLineRecordResolver
 org.apache.ignite.igfs.mapreduce.records.IgfsStringDelimiterRecordResolver
+org.apache.ignite.internal.DuplicateTypeIdException
 org.apache.ignite.internal.ExecutorAwareMessage
 org.apache.ignite.internal.GridClosureCallMode
 org.apache.ignite.internal.GridComponent$DiscoveryDataExchangeType
@@ -310,6 +311,7 @@ org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask$1
 org.apache.ignite.internal.jdbc2.JdbcQueryTask$QueryResult
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2
 org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery
 org.apache.ignite.internal.managers.GridManagerAdapter$1$1
 org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager$CheckpointSet
@@ -341,7 +343,6 @@ org.apache.ignite.internal.mem.IgniteOutOfMemoryException
 org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl$Segment
 org.apache.ignite.internal.pagemem.wal.StorageException
 org.apache.ignite.internal.pagemem.wal.WALIterator
-org.apache.ignite.internal.pagemem.wal.record.TxRecord$TxAction
 org.apache.ignite.internal.pagemem.wal.record.WALRecord$RecordType
 org.apache.ignite.internal.pagemem.wal.record.delta.DeltaApplicationException
 org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion
@@ -569,7 +570,6 @@ org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$CacheD
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy
 org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl
 org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl$1
-org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl$1$1
 org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl$2
 org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl$3
 org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl$4
@@ -585,6 +585,7 @@ org.apache.ignite.internal.processors.cache.StoredCacheData
 org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy
 org.apache.ignite.internal.processors.cache.binary.BinaryMetadataHolder
 org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey
+org.apache.ignite.internal.processors.cache.binary.BinaryMetadataTransport$2
 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$3
 org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage
 org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage
@@ -905,6 +906,7 @@ org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogMan
 org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException
 org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException
 org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneWalRecordsIterator
+org.apache.ignite.internal.processors.cache.query.CacheQueryEntry
 org.apache.ignite.internal.processors.cache.query.CacheQueryType
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$1
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$1
@@ -919,8 +921,7 @@ org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$10
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$11
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$13$1
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$5
@@ -935,9 +936,9 @@ org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$Metadata
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$MetadataJob$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$MetadataJob$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$MetadataJob$3
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$PeekValueExpiryAwareIterator
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$RequestFutureMap
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$RequestFutureMap$1
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$ScanQueryIterator
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryMetricsAdapter
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryMetricsKey
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest
@@ -1032,9 +1033,9 @@ org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry
 org.apache.ignite.internal.processors.cache.version.GridCacheVersion
 org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext$State
 org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx
-org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl$UserCacheObjectByteArrayImpl
-org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl$UserCacheObjectImpl
-org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl$UserKeyCacheObjectImpl
+org.apache.ignite.internal.processors.cacheobject.UserCacheObjectByteArrayImpl
+org.apache.ignite.internal.processors.cacheobject.UserCacheObjectImpl
+org.apache.ignite.internal.processors.cacheobject.UserKeyCacheObjectImpl
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1MLA
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C2
@@ -1264,6 +1265,7 @@ org.apache.ignite.internal.processors.marshaller.MappingProposedMessage$Proposal
 org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem
 org.apache.ignite.internal.processors.marshaller.MissingMappingRequestMessage
 org.apache.ignite.internal.processors.marshaller.MissingMappingResponseMessage
+org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType
 org.apache.ignite.internal.processors.odbc.odbc.escape.OdbcEscapeType
 org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure
 org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate
@@ -1287,6 +1289,7 @@ org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQue
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryRemoteFilter
+org.apache.ignite.internal.processors.platform.client.IgniteClientException
 org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilter
 org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilterImpl
 org.apache.ignite.internal.processors.platform.compute.PlatformAbstractJob
@@ -1346,6 +1349,7 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$8
 org.apache.ignite.internal.processors.query.GridQueryProcessor$9
 org.apache.ignite.internal.processors.query.GridQueryProcessor$SchemaOperation$1
 org.apache.ignite.internal.processors.query.IgniteSQLException
+org.apache.ignite.internal.processors.query.QueryField
 org.apache.ignite.internal.processors.query.QueryIndexKey
 org.apache.ignite.internal.processors.query.QuerySchema
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest
@@ -1359,7 +1363,9 @@ org.apache.ignite.internal.processors.query.schema.message.SchemaAbstractDiscove
 org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage
 org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage
 org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage
+org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractAlterTableOperation
 org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation
+org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation
 org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexAbstractOperation
 org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation
 org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 0241165..6712b5b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -27,6 +27,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.eviction.EvictableEntry;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -831,6 +832,11 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** {@inheritDoc} */
+    @Override public CacheObject unswap(CacheDataRow row) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasLockCandidate(long threadId) throws GridCacheEntryRemovedException {
         return localCandidate(threadId) != null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/23104171/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
index 5041690..c378b6f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SpiQuery;
@@ -183,6 +184,46 @@ public class GridCacheQueryTransformerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testGetObjectFieldPartitioned() throws Exception {
+        IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
+
+        Affinity<Integer> affinity = affinity(cache);
+
+        try {
+            int [] keys = new int[50];
+
+            for (int i = 0, j = 0; i < keys.length; j++) {
+                if (affinity.partition(j) == 0)
+                    keys[i++] = j;
+            }
+
+            for (int i : keys)
+                cache.put(i, new Value("str" + i, i * 100));
+
+            IgniteClosure<Cache.Entry<Integer, Value>, Integer> transformer =
+                new IgniteClosure<Cache.Entry<Integer, Value>, Integer>() {
+                    @Override public Integer apply(Cache.Entry<Integer, Value> e) {
+                        return e.getValue().idx;
+                    }
+                };
+
+            List<Integer> res = cache.query(new ScanQuery<Integer, Value>().setPartition(0), transformer).getAll();
+
+            assertEquals(50, res.size());
+
+            Collections.sort(res);
+
+            for (int i = 0; i < keys.length; i++)
+                assertEquals(keys[i] * 100, res.get(i).intValue());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetObjectFieldFiltered() throws Exception {
         IgniteCache<Integer, Value> cache = grid().createCache("test-cache");
 


[13/50] [abbrv] ignite git commit: IGNITE-6250 .NET: Thin client: Basic exception handling

Posted by yz...@apache.org.
IGNITE-6250 .NET: Thin client: Basic exception handling

This closes #2698


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: bd60ab1fa6aaf433849e8a052af41a2a333c7904
Parents: 1a64615
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Sep 20 16:49:49 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Wed Sep 20 18:12:58 2017 +0300

----------------------------------------------------------------------
 .../odbc/ClientListenerNioListener.java         |   2 +-
 .../odbc/ClientListenerRequestHandler.java      |   3 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |   2 +-
 .../odbc/odbc/OdbcRequestHandler.java           |   2 +-
 .../client/ClientConnectionContext.java         |   4 +-
 .../platform/client/ClientMessageParser.java    |  20 ++-
 .../platform/client/ClientRawRequest.java       |  45 +++++++
 .../platform/client/ClientRequest.java          |   9 ++
 .../platform/client/ClientRequestHandler.java   |  17 +--
 .../platform/client/ClientResponse.java         |  33 ++++-
 .../platform/client/ClientStatus.java           |  45 +++++++
 .../platform/client/IgniteClientException.java  |  65 ++++++++++
 .../client/cache/ClientCacheRequest.java        |  11 +-
 .../Client/Cache/CacheTest.cs                   |  20 +++
 .../Client/Cache/ScanQueryTest.cs               |  34 ++++-
 .../Client/ClientConnectionTest.cs              |   5 +-
 .../Client/RawSocketTest.cs                     | 130 +++++++++++++------
 .../Apache.Ignite.Core.csproj                   |   2 +
 .../Client/IgniteClientException.cs             | 108 +++++++++++++++
 .../Impl/Client/Cache/CacheClient.cs            |  23 +++-
 .../Impl/Client/ClientSocket.cs                 |  21 +--
 .../Impl/Client/ClientStatus.cs                 |  31 +++++
 22 files changed, 546 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
index f0f4903..faecab3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
@@ -154,7 +154,7 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<byte
         catch (Exception e) {
             log.error("Failed to process client request [req=" + req + ']', e);
 
-            ses.send(parser.encode(handler.handleException(e)));
+            ses.send(parser.encode(handler.handleException(e, req)));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
index e28d0d9..cebde08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
@@ -35,9 +35,10 @@ public interface ClientListenerRequestHandler {
      * Handle exception.
      *
      * @param e Exception.
+     * @param req Request.
      * @return Error response.
      */
-    public ClientListenerResponse handleException(Exception e);
+    public ClientListenerResponse handleException(Exception e, ClientListenerRequest req);
 
     /**
      * Write successful handshake response.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 2143fb6f..ea25b11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -188,7 +188,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public ClientListenerResponse handleException(Exception e) {
+    @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
         return exceptionToResult(e);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index cca991d..fd31f39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -160,7 +160,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public ClientListenerResponse handleException(Exception e) {
+    @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
         return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, e.toString());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index 7bab76f..2d81f35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform.client;
 
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
@@ -123,7 +122,8 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
         long curCnt0 = curCnt.get();
 
         if (curCnt0 >= maxCursors) {
-            throw new IgniteException("Too many open cursors (either close other open cursors or increase the " +
+            throw new IgniteClientException(ClientStatus.TOO_MANY_CURSORS,
+                "Too many open cursors (either close other open cursors or increase the " +
                 "limit through ClientConnectorConfiguration.maxOpenCursorsPerConnection) [maximum=" + maxCursors +
                 ", current=" + curCnt0 + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
index 219afdc..84d3eee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform.client;
 
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
@@ -29,10 +28,10 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProce
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
-import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeNameGetRequest;
 import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeGetRequest;
-import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypePutRequest;
+import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeNameGetRequest;
 import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeNamePutRequest;
+import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypePutRequest;
 import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheGetRequest;
 import org.apache.ignite.internal.processors.platform.client.cache.ClientCachePutRequest;
 import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheScanQueryNextPageRequest;
@@ -91,6 +90,16 @@ public class ClientMessageParser implements ClientListenerMessageParser {
         BinaryInputStream inStream = new BinaryHeapInputStream(msg);
         BinaryRawReaderEx reader = marsh.reader(inStream);
 
+        return decode(reader);
+    }
+
+    /**
+     * Decodes the request.
+     *
+     * @param reader Reader.
+     * @return Request.
+     */
+    public ClientListenerRequest decode(BinaryRawReaderEx reader) {
         short opCode = reader.readShort();
 
         switch (opCode) {
@@ -122,11 +131,14 @@ public class ClientMessageParser implements ClientListenerMessageParser {
                 return new ClientResourceCloseRequest(reader);
         }
 
-        throw new IgniteException("Invalid operation: " + opCode);
+        return new ClientRawRequest(reader.readLong(), ClientStatus.INVALID_OP_CODE,
+            "Invalid request op code: " + opCode);
     }
 
     /** {@inheritDoc} */
     @Override public byte[] encode(ClientListenerResponse resp) {
+        assert resp != null;
+
         BinaryHeapOutputStream outStream = new BinaryHeapOutputStream(32);
 
         BinaryRawWriterEx writer = marsh.writer(outStream);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java
new file mode 100644
index 0000000..e6c12de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client;
+
+/**
+ * Raw request, produces a ClientResponse directly.
+ */
+public class ClientRawRequest extends ClientRequest {
+    /** Status code. */
+    private final int status;
+
+    /** Message. */
+    private final String msg;
+
+    /**
+     * Initializes a new instance of ClientRawRequest class.
+     * @param reqId Request id.
+     * @param msg Message.
+     */
+    public ClientRawRequest(long reqId, int status, String msg) {
+        super(reqId);
+        this.status = status;
+        this.msg = msg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientResponse process(ClientConnectionContext ctx) {
+        return new ClientResponse(requestId(), status, msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
index 788d2e0..76823b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
@@ -36,6 +36,15 @@ public class ClientRequest implements ClientListenerRequest {
         reqId = reader.readLong();
     }
 
+    /**
+     * Constructor.
+     *
+     * @param reqId Request id.
+     */
+    public ClientRequest(long reqId) {
+        this.reqId = reqId;
+    }
+
     /** {@inheritDoc} */
     @Override public long requestId() {
         return reqId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
index 499fb05..3f6c082 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
@@ -42,17 +42,18 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
 
     /** {@inheritDoc} */
     @Override public ClientListenerResponse handle(ClientListenerRequest req) {
-        try {
-            return ((ClientRequest)req).process(ctx);
-        }
-        catch (Throwable e) {
-            return new ClientResponse(req.requestId(), e.getMessage());
-        }
+        return ((ClientRequest) req).process(ctx);
     }
 
     /** {@inheritDoc} */
-    @Override public ClientListenerResponse handleException(Exception e) {
-        return null;
+    @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
+        assert req != null;
+        assert e != null;
+
+        int status = e instanceof IgniteClientException ?
+            ((IgniteClientException)e).statusCode() : ClientStatus.FAILED;
+
+        return new ClientResponse(req.requestId(), status, e.getMessage());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
index a1bc4b6..6bffac3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
@@ -33,7 +33,7 @@ public class ClientResponse extends ClientListenerResponse {
      * @param reqId Request id.
      */
     public ClientResponse(long reqId) {
-        super(STATUS_SUCCESS, null);
+        super(ClientStatus.SUCCESS, null);
 
         this.reqId = reqId;
     }
@@ -42,9 +42,23 @@ public class ClientResponse extends ClientListenerResponse {
      * Constructor.
      *
      * @param reqId Request id.
+     * @param err Error message.
      */
     public ClientResponse(long reqId, String err) {
-        super(STATUS_FAILED, err);
+        super(ClientStatus.FAILED, err);
+
+        this.reqId = reqId;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param reqId Request id.
+     * @param status Status code.
+     * @param err Error message.
+     */
+    public ClientResponse(long reqId, int status, String err) {
+        super(status, err);
 
         this.reqId = reqId;
     }
@@ -54,12 +68,19 @@ public class ClientResponse extends ClientListenerResponse {
      */
     public void encode(BinaryRawWriterEx writer) {
         writer.writeLong(reqId);
+        writer.writeInt(status());
 
-        if (status() == STATUS_SUCCESS) {
-            writer.writeBoolean(true);
-        } else {
-            writer.writeBoolean(false);
+        if (status() != ClientStatus.SUCCESS) {
             writer.writeString(error());
         }
     }
+
+    /**
+     * Gets the request id.
+     *
+     * @return Request id.
+     */
+    public long requestId() {
+        return reqId;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
new file mode 100644
index 0000000..0f374ff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client;
+
+/**
+ * Client status codes.
+ */
+public final class ClientStatus {
+    /**
+     * No-op constructor to prevent instantiation.
+     */
+    private ClientStatus (){
+        // No-op.
+    }
+
+    /** Command succeeded. */
+    public static final int SUCCESS = 0;
+
+    /** Command failed. */
+    public static final int FAILED = 1;
+
+    /** Invalid op code. */
+    public static final int INVALID_OP_CODE = 2;
+
+    /** Cache does not exist. */
+    public static final int CACHE_DOES_NOT_EXIST = 1000;
+
+    /** Too many cursors. */
+    public static final int TOO_MANY_CURSORS = 1010;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java
new file mode 100644
index 0000000..9bcabab
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java
@@ -0,0 +1,65 @@
+/*
+ * 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.platform.client;
+
+import org.apache.ignite.IgniteException;
+
+import java.sql.SQLException;
+
+/**
+ * Client exception.
+ */
+public class IgniteClientException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Code to return as {@link SQLException#vendorCode} */
+    private final int statusCode;
+
+    /**
+     * Constructor.
+     *
+     * @param statusCode Status code (see {@link ClientStatus}).
+     * @param msg Message.
+     */
+    public IgniteClientException(int statusCode, String msg) {
+        this(statusCode, msg, null);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param statusCode Status code (see {@link ClientStatus}).
+     * @param msg Message.
+     * @param cause Cause.
+     */
+    public IgniteClientException(int statusCode, String msg, Exception cause) {
+        super(msg, cause);
+
+        this.statusCode = statusCode;
+    }
+
+    /**
+     * Gets the status code.
+     *
+     * @return Status code.
+     */
+    public int statusCode() {
+        return statusCode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
index fd7cc6c..8f81e94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
@@ -19,8 +19,11 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
 import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 
 /**
  * Cache get request.
@@ -74,7 +77,13 @@ class ClientCacheRequest extends ClientRequest {
      * @return Cache.
      */
     protected IgniteCache rawCache(ClientConnectionContext ctx) {
-        String cacheName = ctx.kernalContext().cache().context().cacheContext(cacheId).cache().name();
+        GridCacheContext<Object, Object> cacheCtx = ctx.kernalContext().cache().context().cacheContext(cacheId);
+
+        if (cacheCtx == null)
+            throw new IgniteClientException(ClientStatus.CACHE_DOES_NOT_EXIST, "Cache does not exist [cacheId= " +
+                cacheId + "]", null);
+
+        String cacheName = cacheCtx.cache().name();
 
         return ctx.kernalContext().grid().cache(cacheName);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
index 6cc0662..4293c5b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     using System.Threading;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Client;
+    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>
@@ -151,5 +152,24 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
 
             clients.ToList().ForEach(x => x.Value.Dispose());
         }
+
+        /// <summary>
+        /// Tests the cache exceptions.
+        /// </summary>
+        [Test]
+        public void TestExceptions()
+        {
+            using (var client = GetClient())
+            {
+                // Getting the cache instance does not throw.
+                var cache = client.GetCache<int, int>("foobar");
+
+                // Accessing non-existent cache throws.
+                var ex = Assert.Throws<IgniteClientException>(() => cache.Put(1, 1));
+
+                Assert.AreEqual("Cache doesn't exist: foobar", ex.Message);
+                Assert.AreEqual((int) ClientStatus.CacheDoesNotExist, ex.ErrorCode);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/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 7f8b589..17a2b3f 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
@@ -24,8 +24,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     using System.Linq;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Query;
-    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Configuration;
+    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>
@@ -135,6 +136,28 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         }
 
         /// <summary>
+        /// Tests the exception in filter.
+        /// </summary>
+        [Test]
+        public void TestExceptionInFilter()
+        {
+            GetPersonCache();
+
+            using (var client = GetClient())
+            {
+                var clientCache = client.GetCache<int, Person>(CacheName);
+
+                var qry = new ScanQuery<int, Person>(new PersonFilter(x =>
+                {
+                    throw new ArithmeticException("foo");
+                }));
+
+                var ex = Assert.Throws<IgniteClientException>(() => clientCache.Query(qry).GetAll());
+                Assert.AreEqual("foo", ex.Message);
+            }
+        }
+
+        /// <summary>
         /// Tests multiple cursors with the same client.
         /// </summary>
         [Test]
@@ -154,8 +177,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
                 var cur3 = clientCache.Query(qry).GetEnumerator();
 
                 // MaxCursors = 3
-                var ex = Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                var ex = Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
                 Assert.AreEqual("Too many open cursors", ex.Message.Substring(0, 21));
+                Assert.AreEqual((int) ClientStatus.TooManyCursors, ex.ErrorCode);
 
                 var count = 0;
 
@@ -177,17 +201,17 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
                 var c2 = clientCache.Query(qry);
                 var c3 = clientCache.Query(qry);
 
-                Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
 
                 // Close one of the cursors.
                 c1.Dispose();
                 c1 = clientCache.Query(qry);
-                Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
 
                 // Close cursor via GetAll.
                 c1.GetAll();
                 c1 = clientCache.Query(qry);
-                Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
 
                 c1.Dispose();
                 c2.Dispose();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 7dcec55..66aa844 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Tests.Client
     using System.Net;
     using System.Net.Sockets;
     using Apache.Ignite.Core.Client;
-    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
@@ -119,9 +118,11 @@ namespace Apache.Ignite.Core.Tests.Client
             using (Ignition.Start(TestUtils.GetTestConfiguration()))
             {
                 // ReSharper disable once ObjectCreationAsStatement
-                var ex = Assert.Throws<IgniteException>(() => new ClientSocket(GetClientConfiguration(),
+                var ex = Assert.Throws<IgniteClientException>(() => new ClientSocket(GetClientConfiguration(),
                     new ClientProtocolVersion(-1, -1, -1)));
 
+                Assert.AreEqual((int) ClientStatus.Fail, ex.ErrorCode);
+
                 Assert.AreEqual("Client handhsake failed: 'Unsupported version.'. " +
                                 "Client version: -1.-1.-1. Server version: 1.0.0", ex.Message);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
index b34e037..b637e88 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
@@ -20,17 +20,17 @@ namespace Apache.Ignite.Core.Tests.Client
     using System;
     using System.Net;
     using System.Net.Sockets;
-    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>
     /// Tests the thin client mode with a raw socket.
     /// </summary>
-    public class RawSocketTest
+    public class RawSocketTest : ClientTestBase
     {
         /// <summary>
         /// Tests the socket handshake connection.
@@ -38,59 +38,113 @@ namespace Apache.Ignite.Core.Tests.Client
         [Test]
         public void TestCacheGet()
         {
-            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            var ignite = Ignition.GetIgnite();
+
+            var marsh = ((Ignite) ignite).Marshaller;
+
+            // Create cache.
+            var cache = GetCache<string>();
+            cache[1] = "bar";
+
+            // Connect socket.
+            var sock = GetSocket();
+
+            // Cache get.
+            SendRequest(sock, stream =>
             {
-                ClientConnectorConfiguration = new ClientConnectorConfiguration()
-            };
+                stream.WriteShort(1); // OP_GET
+                stream.WriteLong(1); // Request id.
+                var cacheId = BinaryUtils.GetStringHashCode(cache.Name);
+                stream.WriteInt(cacheId);
+                stream.WriteByte(0); // Flags (withSkipStore, etc)
+
+                var writer = marsh.StartMarshal(stream);
 
-            using (var ignite = Ignition.Start(cfg))
+                writer.WriteObject(1); // Key
+            });
+
+            var msg = ReceiveMessage(sock);
+
+            using (var stream = new BinaryHeapStream(msg))
             {
-                var marsh = ((Ignite) ignite).Marshaller;
+                var reader = marsh.StartUnmarshal(stream);
 
-                // Create cache.
-                var cacheCfg = new CacheConfiguration("foo", new QueryEntity(typeof(int), typeof(string)));
-                var cache = ignite.CreateCache<int, string>(cacheCfg);
-                cache[1] = "bar";
+                var requestId = reader.ReadLong();
+                Assert.AreEqual(1, requestId);
 
-                // Connect socket.
-                var sock = GetSocket(ClientConnectorConfiguration.DefaultPort);
-                Assert.IsTrue(sock.Connected);
+                var status = reader.ReadInt();
+                Assert.AreEqual(0, status); // Success.
 
-                DoHandshake(sock);
+                var res = reader.ReadObject<string>();
+                Assert.AreEqual(cache[1], res);
+            }
+        }
 
-                // Cache get.
-                SendRequest(sock, stream =>
-                {
-                    stream.WriteShort(1);  // OP_GET
-                    stream.WriteLong(1);  // Request id.
-                    var cacheId = BinaryUtils.GetStringHashCode(cache.Name);
-                    stream.WriteInt(cacheId);
-                    stream.WriteByte(0);  // Flags (withSkipStore, etc)
+        /// <summary>
+        /// Tests invalid operation code.
+        /// </summary>
+        [Test]
+        public void TestInvalidOpCode()
+        {
+            // Connect socket.
+            var sock = GetSocket();
 
-                    var writer = marsh.StartMarshal(stream);
+            // Request invalid operation.
+            SendRequest(sock, stream =>
+            {
+                stream.WriteShort(-1);
+                stream.WriteLong(11);  // Request id.
+            });
 
-                    writer.WriteObject(1);  // Key
-                });
+            var msg = ReceiveMessage(sock);
 
-                var msg = ReceiveMessage(sock);
-                
-                using (var stream = new BinaryHeapStream(msg))
-                {
-                    var reader = marsh.StartUnmarshal(stream);
+            using (var stream = new BinaryHeapStream(msg))
+            {
+                var reader = BinaryUtils.Marshaller.StartUnmarshal(stream);
 
-                    var requestId = reader.ReadLong();
-                    Assert.AreEqual(1, requestId);
+                var requestId = reader.ReadLong();
+                Assert.AreEqual(11, requestId);
 
-                    var success = reader.ReadBoolean();
-                    Assert.IsTrue(success);
+                var status = reader.ReadInt();
+                Assert.AreEqual((int) ClientStatus.InvalidOpCode, status);
 
-                    var res = reader.ReadObject<string>();
-                    Assert.AreEqual(cache[1], res);
-                }
+                var err = reader.ReadObject<string>();
+                Assert.AreEqual("Invalid request op code: -1", err);
             }
         }
 
         /// <summary>
+        /// Tests invalid message (can't be parsed).
+        /// </summary>
+        [Test]
+        public void TestInvalidMessage()
+        {
+            // Connect socket.
+            var sock = GetSocket();
+
+            // Request invalid operation.
+            SendRequest(sock, stream => stream.WriteShort(-1));
+
+            var msg = ReceiveMessage(sock);
+
+            Assert.AreEqual(0, msg.Length);
+        }
+
+        /// <summary>
+        /// Gets the socket.
+        /// </summary>
+        /// <returns>Connected socket after handshake.</returns>
+        private static Socket GetSocket()
+        {
+            var sock = GetSocket(ClientConnectorConfiguration.DefaultPort);
+            Assert.IsTrue(sock.Connected);
+
+            DoHandshake(sock);
+
+            return sock;
+        }
+
+        /// <summary>
         /// Does the handshake.
         /// </summary>
         /// <param name="sock">The sock.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 75da6c8..28fe741 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -99,6 +99,7 @@
     <Compile Include="Cache\IMemoryMetrics.cs" />
     <Compile Include="Client\Cache\ICacheClient.cs" />
     <Compile Include="Client\IgniteClientConfiguration.cs" />
+    <Compile Include="Client\IgniteClientException.cs" />
     <Compile Include="Client\IIgniteClient.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
@@ -109,6 +110,7 @@
     <Compile Include="Impl\Cache\Query\PlatformQueryQursorBase.cs" />
     <Compile Include="Impl\Binary\BinaryProcessorClient.cs" />
     <Compile Include="Impl\Binary\IBinaryProcessor.cs" />
+    <Compile Include="Impl\Client\ClientStatus.cs" />
     <Compile Include="Impl\IIgniteInternal.cs" />
     <Compile Include="Impl\Client\Cache\CacheClient.cs" />
     <Compile Include="Impl\Client\ClientOp.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs
new file mode 100644
index 0000000..a20bec3
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Client
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Client;
+
+    /// <summary>
+    /// Ignite thin client exception.
+    /// </summary>
+    [Serializable]
+    public class IgniteClientException : IgniteException
+    {
+        /** Error code field. */
+        private const string ErrorCodeField = "ErrorCode";
+
+        /** Error code. */
+        private readonly int _errorCode = (int) ClientStatus.Fail;
+
+        /// <summary>
+        /// Gets the error code.
+        /// </summary>
+        public int ErrorCode
+        {
+            get { return _errorCode; }
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException"/> class.
+        /// </summary>
+        public IgniteClientException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException" /> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public IgniteClientException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException" /> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public IgniteClientException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException" /> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        /// <param name="errorCode">The error code.</param>
+        public IgniteClientException(string message, Exception cause, int errorCode) : base(message, cause)
+        {
+            _errorCode = errorCode;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected IgniteClientException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            _errorCode = info.GetInt32(ErrorCodeField);
+        }
+
+        /// <summary>
+        /// When overridden in a derived class, sets the <see cref="SerializationInfo" /> 
+        /// with information about the exception.
+        /// </summary>
+        /// <param name="info">The <see cref="SerializationInfo" /> that holds the serialized object data
+        /// about the exception being thrown.</param>
+        /// <param name="context">The <see cref="StreamingContext" /> that contains contextual information
+        /// about the source or destination.</param>
+        public override void GetObjectData(SerializationInfo info, StreamingContext context)
+        {
+            base.GetObjectData(info, context);
+
+            info.AddValue(ErrorCodeField, _errorCode);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
index 974bab3..5492ef8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
@@ -79,12 +79,6 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         }
 
         /** <inheritDoc /> */
-        public IIgniteClient Ignite
-        {
-            get { return _ignite; }
-        }
-
-        /** <inheritDoc /> */
         public TV this[TK key]
         {
             get { return Get(key); }
@@ -142,7 +136,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
 
                     _marsh.FinishMarshal(writer);
                 }
-            }, readFunc);
+            }, readFunc, HandleError<T>);
         }
 
         /// <summary>
@@ -199,6 +193,21 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         }
 
         /// <summary>
+        /// Handles the error.
+        /// </summary>
+        private T HandleError<T>(ClientStatus status, string msg)
+        {
+            switch (status)
+            {
+                case ClientStatus.CacheDoesNotExist:
+                    throw new IgniteClientException("Cache doesn't exist: " + Name, null, (int) status);
+
+                default:
+                    throw new IgniteClientException(msg, null, (int) status);
+            }
+        }
+
+        /// <summary>
         /// Gets the key not found exception.
         /// </summary>
         private static KeyNotFoundException GetKeyNotFoundException()

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index 1a245a1..4eb53dd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -68,7 +68,7 @@ namespace Apache.Ignite.Core.Impl.Client
         /// Performs a send-receive operation.
         /// </summary>
         public T DoOutInOp<T>(ClientOp opId, Action<IBinaryStream> writeAction,
-            Func<IBinaryStream, T> readFunc)
+            Func<IBinaryStream, T> readFunc, Func<ClientStatus, string, T> errorFunc = null)
         {
             var requestId = Interlocked.Increment(ref _requestId);
 
@@ -88,19 +88,22 @@ namespace Apache.Ignite.Core.Impl.Client
                 var resRequestId = stream.ReadLong();
                 Debug.Assert(requestId == resRequestId);
 
-                if (!stream.ReadBool())
+                var statusCode = (ClientStatus) stream.ReadInt();
+
+                if (statusCode == ClientStatus.Success)
                 {
-                    // Error.
-                    throw new IgniteException(BinaryUtils.Marshaller.StartUnmarshal(stream).ReadString());
+                    return readFunc != null ? readFunc(stream) : default(T);
                 }
 
-                if (readFunc != null)
+                var msg = BinaryUtils.Marshaller.StartUnmarshal(stream).ReadString();
+
+                if (errorFunc != null)
                 {
-                    return readFunc(stream);
+                    return errorFunc(statusCode, msg);
                 }
-            }
 
-            return default(T);
+                throw new IgniteClientException(msg, null, (int) statusCode);
+            }
         }
 
         /// <summary>
@@ -136,7 +139,7 @@ namespace Apache.Ignite.Core.Impl.Client
 
                 var errMsg = BinaryUtils.Marshaller.Unmarshal<string>(stream);
 
-                throw new IgniteException(string.Format(
+                throw new IgniteClientException(string.Format(
                     "Client handhsake failed: '{0}'. Client version: {1}. Server version: {2}",
                     errMsg, version, serverVersion));
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs
new file mode 100644
index 0000000..a2e66e8
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Client
+{
+    /// <summary>
+    /// Client status codes.
+    /// </summary>
+    internal enum ClientStatus
+    {
+        Success = 0,
+        Fail = 1,
+        InvalidOpCode = 2,
+        CacheDoesNotExist = 1000,
+        TooManyCursors = 1010
+    }
+}


[26/50] [abbrv] ignite git commit: IGNITE-6294: ODBC: implemented SQLSTATE management. This closes #2716.

Posted by yz...@apache.org.
IGNITE-6294: ODBC: implemented SQLSTATE management. This closes #2716.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: fc34091c750cdaf759a54a989a84c76f8d9f9f7f
Parents: 49a3631
Author: Igor Sapego <ig...@gmail.com>
Authored: Thu Sep 21 20:02:33 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Thu Sep 21 20:05:40 2017 +0300

----------------------------------------------------------------------
 .../odbc/odbc/OdbcConnectionContext.java        |   6 +-
 .../processors/odbc/odbc/OdbcMessageParser.java |  17 +-
 .../odbc/odbc/OdbcQueryExecuteBatchResult.java  |  16 +-
 .../odbc/odbc/OdbcRequestHandler.java           |  54 +++-
 .../processors/odbc/odbc/OdbcUtils.java         |  24 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   1 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   1 +
 .../project/vs/odbc-test.vcxproj.filters        |   3 +
 .../platforms/cpp/odbc-test/src/errors_test.cpp | 289 +++++++++++++++++++
 .../platforms/cpp/odbc-test/src/parser_test.cpp |   4 +-
 .../cpp/odbc-test/src/queries_test.cpp          |  16 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   1 +
 .../cpp/odbc/include/ignite/odbc/common_types.h | 131 ++++++++-
 .../ignite/odbc/diagnostic/diagnosable.h        |   9 +
 .../odbc/diagnostic/diagnosable_adapter.h       |   8 +
 .../cpp/odbc/include/ignite/odbc/message.h      |  48 +--
 .../cpp/odbc/include/ignite/odbc/odbc_error.h   |  94 ++++++
 .../cpp/odbc/include/ignite/odbc/parser.h       |  35 ++-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   1 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   3 +
 modules/platforms/cpp/odbc/src/common_types.cpp |  51 ++++
 modules/platforms/cpp/odbc/src/connection.cpp   |  25 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   8 +
 .../odbc/src/diagnostic/diagnostic_record.cpp   | 105 ++++++-
 modules/platforms/cpp/odbc/src/message.cpp      |  37 ++-
 .../cpp/odbc/src/query/batch_query.cpp          |  13 +-
 .../odbc/src/query/column_metadata_query.cpp    |  11 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  31 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  11 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  11 +-
 30 files changed, 960 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
index f2450e2..a4af478 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
@@ -32,10 +32,10 @@ import org.apache.ignite.internal.util.GridSpinBusyLock;
  */
 public class OdbcConnectionContext implements ClientListenerConnectionContext {
     /** Version 2.1.0. */
-    private static final ClientListenerProtocolVersion VER_2_1_0 = ClientListenerProtocolVersion.create(2, 1, 0);
+    public static final ClientListenerProtocolVersion VER_2_1_0 = ClientListenerProtocolVersion.create(2, 1, 0);
 
     /** Version 2.1.5: added "lazy" flag. */
-    private static final ClientListenerProtocolVersion VER_2_1_5 = ClientListenerProtocolVersion.create(2, 1, 5);
+    public static final ClientListenerProtocolVersion VER_2_1_5 = ClientListenerProtocolVersion.create(2, 1, 5);
 
     /** Current version. */
     private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_1_5;
@@ -101,7 +101,7 @@ public class OdbcConnectionContext implements ClientListenerConnectionContext {
         handler = new OdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins,
                 enforceJoinOrder, replicatedOnly, collocated, lazy);
 
-        parser = new OdbcMessageParser(ctx);
+        parser = new OdbcMessageParser(ctx, ver);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
index 2ab7cee..04e2e25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
+import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
@@ -51,11 +52,16 @@ public class OdbcMessageParser implements ClientListenerMessageParser {
     /** Logger. */
     private final IgniteLogger log;
 
+    /** Protocol version */
+    private final ClientListenerProtocolVersion ver;
+
     /**
      * @param ctx Context.
+     * @param ver Protocol version.
      */
-    public OdbcMessageParser(GridKernalContext ctx) {
+    public OdbcMessageParser(GridKernalContext ctx, ClientListenerProtocolVersion ver) {
         this.ctx = ctx;
+        this.ver = ver;
 
         log = ctx.log(getClass());
 
@@ -194,7 +200,11 @@ public class OdbcMessageParser implements ClientListenerMessageParser {
             BinaryThreadLocalContext.get().schemaHolder(), null);
 
         // Writing status.
-        writer.writeByte((byte) msg.status());
+        if (ver.compareTo(OdbcConnectionContext.VER_2_1_5) < 0) {
+            writer.writeByte((byte) (msg.status() == ClientListenerResponse.STATUS_SUCCESS ?
+                ClientListenerResponse.STATUS_SUCCESS : ClientListenerResponse.STATUS_FAILED));
+        } else
+            writer.writeInt(msg.status());
 
         if (msg.status() != ClientListenerResponse.STATUS_SUCCESS) {
             writer.writeString(msg.error());
@@ -234,6 +244,9 @@ public class OdbcMessageParser implements ClientListenerMessageParser {
             if (res.errorMessage() != null) {
                 writer.writeLong(res.errorSetIdx());
                 writer.writeString(res.errorMessage());
+
+                if (ver.compareTo(OdbcConnectionContext.VER_2_1_5) >= 0)
+                    writer.writeInt(res.errorCode());
             }
         }
         else if (res0 instanceof OdbcQueryFetchResult) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchResult.java
index ee02fd6..c8f61dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchResult.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.odbc.odbc;
 
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -29,6 +30,9 @@ public class OdbcQueryExecuteBatchResult {
     /** Index of the set which caused an error. */
     private final long errorSetIdx;
 
+    /** Error code. */
+    private final int errorCode;
+
     /** Error message. */
     private final String errorMessage;
 
@@ -39,17 +43,20 @@ public class OdbcQueryExecuteBatchResult {
         this.rowsAffected = rowsAffected;
         this.errorSetIdx = -1;
         this.errorMessage = null;
+        this.errorCode = ClientListenerResponse.STATUS_SUCCESS;
     }
 
     /**
      * @param rowsAffected Number of rows affected by the query.
      * @param errorSetIdx Sets processed.
+     * @param errorCode Error code.
      * @param errorMessage Error message.
      */
-    public OdbcQueryExecuteBatchResult(long rowsAffected, long errorSetIdx, String errorMessage) {
+    public OdbcQueryExecuteBatchResult(long rowsAffected, long errorSetIdx, int errorCode, String errorMessage) {
         this.rowsAffected = rowsAffected;
         this.errorSetIdx = errorSetIdx;
         this.errorMessage = errorMessage;
+        this.errorCode = errorCode;
     }
 
     /**
@@ -72,4 +79,11 @@ public class OdbcQueryExecuteBatchResult {
     @Nullable public String errorMessage() {
         return errorMessage;
     }
+
+    /**
+     * @return Error code.
+     */
+    public int errorCode() {
+        return errorCode;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index fd31f39..07b41f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
@@ -42,6 +43,7 @@ import org.apache.ignite.internal.processors.odbc.odbc.escape.OdbcEscapeUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -125,7 +127,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         OdbcRequest req = (OdbcRequest)req0;
 
         if (!busyLock.enterBusy())
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED,
+            return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN,
                     "Failed to handle ODBC request because node is stopping: " + req);
 
         try {
@@ -152,7 +154,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
                     return getParamsMeta((OdbcQueryGetParamsMetaRequest)req);
             }
 
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
+            return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Unsupported ODBC request: " + req);
         }
         finally {
             busyLock.leaveBusy();
@@ -161,7 +163,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
 
     /** {@inheritDoc} */
     @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
-        return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, e.toString());
+        return exceptionToResult(e);
     }
 
     /** {@inheritDoc} */
@@ -219,7 +221,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         int cursorCnt = qryCursors.size();
 
         if (maxCursors > 0 && cursorCnt >= maxCursors)
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, "Too many open cursors (either close " +
+            return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Too many open cursors (either close " +
                 "other open cursors or increase the limit through " +
                 "ClientConnectorConfiguration.maxOpenCursorsPerConnection) [maximum=" + maxCursors +
                 ", current=" + cursorCnt + ']');
@@ -258,7 +260,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
 
             U.error(log, "Failed to execute SQL query [reqId=" + req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, OdbcUtils.retrieveH2ErrorMessage(e));
+            return exceptionToResult(e);
         }
     }
 
@@ -308,10 +310,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to execute SQL query [reqId=" + req.requestId() + ", req=" + req + ']', e);
 
-            OdbcQueryExecuteBatchResult res = new OdbcQueryExecuteBatchResult(rowsAffected, currentSet,
-                    OdbcUtils.retrieveH2ErrorMessage(e));
-
-            return new OdbcResponse(res);
+            return exceptionToBatchResult(e, rowsAffected, currentSet);
         }
     }
 
@@ -364,7 +363,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
             IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(queryId);
 
             if (tuple == null)
-                return new OdbcResponse(ClientListenerResponse.STATUS_FAILED,
+                return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN,
                     "Failed to find query with ID: " + queryId);
 
             CloseCursor(tuple, queryId);
@@ -378,7 +377,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
 
             U.error(log, "Failed to close SQL query [reqId=" + req.requestId() + ", req=" + queryId + ']', e);
 
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, OdbcUtils.retrieveH2ErrorMessage(e));
+            return exceptionToResult(e);
         }
     }
 
@@ -427,7 +426,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to fetch SQL query result [reqId=" + req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, OdbcUtils.retrieveH2ErrorMessage(e));
+            return exceptionToResult(e);
         }
     }
 
@@ -492,7 +491,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get columns metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, OdbcUtils.retrieveH2ErrorMessage(e));
+            return exceptionToResult(e);
         }
     }
 
@@ -540,7 +539,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get tables metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, OdbcUtils.retrieveH2ErrorMessage(e));
+            return exceptionToResult(e);
         }
     }
 
@@ -571,7 +570,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get params metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, OdbcUtils.retrieveH2ErrorMessage(e));
+            return exceptionToResult(e);
         }
     }
 
@@ -681,4 +680,29 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
         return str != null && (F.isEmpty(ptrn) ||
             str.toUpperCase().matches(ptrn.toUpperCase().replace("%", ".*").replace("_", ".")));
     }
+
+    /**
+     * Create {@link OdbcResponse} bearing appropriate Ignite specific result code if possible
+     *     from given {@link Exception}.
+     *
+     * @param e Exception to convert.
+     * @return resulting {@link OdbcResponse}.
+     */
+    private OdbcResponse exceptionToBatchResult(Exception e, long rowsAffected, long currentSet) {
+        OdbcQueryExecuteBatchResult res = new OdbcQueryExecuteBatchResult(rowsAffected, currentSet,
+            OdbcUtils.tryRetrieveSqlErrorCode(e), OdbcUtils.tryRetrieveH2ErrorMessage(e));
+
+        return new OdbcResponse(res);
+    }
+
+    /**
+     * Create {@link OdbcResponse} bearing appropriate Ignite specific result code if possible
+     *     from given {@link Exception}.
+     *
+     * @param e Exception to convert.
+     * @return resulting {@link OdbcResponse}.
+     */
+    private OdbcResponse exceptionToResult(Exception e) {
+        return new OdbcResponse(OdbcUtils.tryRetrieveSqlErrorCode(e), OdbcUtils.tryRetrieveH2ErrorMessage(e));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
index 3578145..98fa045 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.odbc.odbc;
 
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.odbc.SqlListenerDataTypes;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 
 /**
  * Various ODBC utility methods.
@@ -141,11 +143,29 @@ public class OdbcUtils {
     }
 
     /**
-     * Retrieves error message from exception.
+     * Tries to retrieve SQL error code of the exception. If the exception is not {@link IgniteSQLException} returns
+     * {@link IgniteQueryErrorCode#UNKNOWN}.
+     *
+     * @param err Error to retrieve code from.
+     * @return Error code.
+     */
+    public static int tryRetrieveSqlErrorCode(Throwable err) {
+        int errorCode = IgniteQueryErrorCode.UNKNOWN;
+
+        if (err instanceof IgniteSQLException)
+            errorCode = ((IgniteSQLException) err).statusCode();
+
+        return errorCode;
+    }
+
+    /**
+     * Tries to retrieve H2 engine error message from exception. If the exception is not of type
+     * "org.h2.jdbc.JdbcSQLException" returns original error message.
+     *
      * @param err Exception.
      * @return Error message.
      */
-    public static String retrieveH2ErrorMessage(Throwable err) {
+    public static String tryRetrieveH2ErrorMessage(Throwable err) {
         String msg = err.getMessage();
 
         Throwable e = err.getCause();

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index 69b62f9..5c335cf 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -79,6 +79,7 @@ ignite_odbc_tests_SOURCES = \
     src/sql_outer_join_test.cpp \
     src/api_robustness_test.cpp \
     src/attributes_test.cpp \
+    src/errors_test.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 3977a24..6ee999e 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -169,6 +169,7 @@
     <ClCompile Include="..\..\src\configuration_test.cpp" />
     <ClCompile Include="..\..\src\connection_info_test.cpp" />
     <ClCompile Include="..\..\src\cursor_test.cpp" />
+    <ClCompile Include="..\..\src\errors_test.cpp" />
     <ClCompile Include="..\..\src\meta_queries_test.cpp" />
     <ClCompile Include="..\..\src\queries_test.cpp" />
     <ClCompile Include="..\..\src\parser_test.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 2a0c7b4..abfe303 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -127,6 +127,9 @@
     <ClCompile Include="..\..\src\attributes_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\errors_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc-test/src/errors_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/errors_test.cpp b/modules/platforms/cpp/odbc-test/src/errors_test.cpp
new file mode 100644
index 0000000..bac9fd4
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/errors_test.cpp
@@ -0,0 +1,289 @@
+/*
+ * 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.
+ */
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <vector>
+#include <string>
+#include <algorithm>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/regex.hpp>
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/common/fixed_size_array.h"
+#include "ignite/ignition.h"
+#include "ignite/impl/binary/binary_utils.h"
+#include "ignite/binary/binary_object.h"
+
+#include "test_type.h"
+#include "complex_type.h"
+#include "test_utils.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::query;
+using namespace ignite::common;
+using namespace ignite_test;
+using namespace ignite::binary;
+using namespace ignite::impl::binary;
+using namespace ignite::impl::interop;
+
+using namespace boost::unit_test;
+
+using ignite::impl::binary::BinaryUtils;
+
+/**
+ * Test setup fixture.
+ */
+struct ErrorTestSuiteFixture 
+{
+    /**
+     * Establish connection to node.
+     *
+     * @param connectStr Connection string.
+     */
+    void Connect(const std::string& connectStr)
+    {
+        // Allocate an environment handle
+        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
+
+        BOOST_REQUIRE(env != NULL);
+
+        // We want ODBC 3 support
+        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
+
+        // Allocate a connection handle
+        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+
+        BOOST_REQUIRE(dbc != NULL);
+
+        // Connect string
+        std::vector<SQLCHAR> connectStr0;
+
+        connectStr0.reserve(connectStr.size() + 1);
+        std::copy(connectStr.begin(), connectStr.end(), std::back_inserter(connectStr0));
+
+        SQLCHAR outstr[ODBC_BUFFER_SIZE];
+        SQLSMALLINT outstrlen;
+
+        // Connecting to ODBC server.
+        SQLRETURN ret = SQLDriverConnect(dbc, NULL, &connectStr0[0], static_cast<SQLSMALLINT>(connectStr0.size()),
+            outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+
+        if (!SQL_SUCCEEDED(ret))
+        {
+            Ignition::StopAll(true);
+
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc));
+        }
+
+        // Allocate a statement handle
+        SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+
+        BOOST_REQUIRE(stmt != NULL);
+    }
+
+    void Disconnect()
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Disconneting from the server.
+        SQLDisconnect(dbc);
+
+        // Releasing allocated handles.
+        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
+        SQLFreeHandle(SQL_HANDLE_ENV, env);
+    }
+
+    static Ignite StartAdditionalNode(const char* name)
+    {
+#ifdef IGNITE_TESTS_32
+        return StartNode("queries-test-32.xml", name);
+#else
+        return StartNode("queries-test.xml", name);
+#endif
+    }
+
+    /**
+     * Constructor.
+     */
+    ErrorTestSuiteFixture() :
+        env(NULL),
+        dbc(NULL),
+        stmt(NULL)
+    {
+        // No-op.
+    }
+
+    /**
+     * Destructor.
+     */
+    ~ErrorTestSuiteFixture()
+    {
+        Disconnect();
+
+        Ignition::StopAll(true);
+    }
+
+    /** Frist cache instance. */
+    //Cache<int64_t, TestType> cache;
+
+    /** ODBC Environment. */
+    SQLHENV env;
+
+    /** ODBC Connect. */
+    SQLHDBC dbc;
+
+    /** ODBC Statement. */
+    SQLHSTMT stmt;
+};
+
+BOOST_FIXTURE_TEST_SUITE(ErrorTestSuite, ErrorTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestConnectFail)
+{
+    // Allocate an environment handle
+    SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
+
+    BOOST_REQUIRE(env != NULL);
+
+    // We want ODBC 3 support
+    SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
+
+    // Allocate a connection handle
+    SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+
+    BOOST_REQUIRE(dbc != NULL);
+
+    // Connect string
+    SQLCHAR connectStr[] = "DRIVER={Apache Ignite};ADDRESS=127.0.0.1:9999;SCHEMA=cache";
+
+    SQLCHAR outstr[ODBC_BUFFER_SIZE];
+    SQLSMALLINT outstrlen;
+
+    // Connecting to ODBC server.
+    SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, SQL_NTS,
+        outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    BOOST_CHECK_EQUAL(GetOdbcErrorState(SQL_HANDLE_DBC, dbc), "08001");
+}
+
+BOOST_AUTO_TEST_CASE(TestDuplicateKey)
+{
+    StartAdditionalNode("Node1");
+
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache");
+
+    SQLCHAR insertReq[] = "INSERT INTO TestType(_key, strField) VALUES(1, 'some')";
+
+    SQLRETURN ret;
+
+    ret = SQLExecDirect(stmt, insertReq, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLExecDirect(stmt, insertReq, SQL_NTS);
+
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    BOOST_CHECK_EQUAL(GetOdbcErrorState(SQL_HANDLE_STMT, stmt), "23000");
+}
+
+BOOST_AUTO_TEST_CASE(TestUpdateKey)
+{
+    StartAdditionalNode("Node1");
+
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache");
+
+    SQLCHAR insertReq[] = "INSERT INTO TestType(_key, strField) VALUES(1, 'some')";
+
+    SQLRETURN ret;
+
+    ret = SQLExecDirect(stmt, insertReq, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    SQLCHAR updateReq[] = "UPDATE TestType SET _key=2 WHERE _key=1";
+
+    ret = SQLExecDirect(stmt, updateReq, SQL_NTS);
+
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    BOOST_CHECK_EQUAL(GetOdbcErrorState(SQL_HANDLE_STMT, stmt), "42000");
+}
+
+BOOST_AUTO_TEST_CASE(TestTableNotFound)
+{
+    StartAdditionalNode("Node1");
+
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=PUBLIC");
+
+    SQLCHAR req[] = "DROP TABLE Nonexisting";
+
+    SQLRETURN ret;
+
+    ret = SQLExecDirect(stmt, req, SQL_NTS);
+
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    BOOST_CHECK_EQUAL(GetOdbcErrorState(SQL_HANDLE_STMT, stmt), "42S02");
+}
+
+BOOST_AUTO_TEST_CASE(TestIndexNotFound)
+{
+    StartAdditionalNode("Node1");
+
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=PUBLIC");
+
+    SQLCHAR req[] = "DROP INDEX Nonexisting";
+
+    SQLRETURN ret;
+
+    ret = SQLExecDirect(stmt, req, SQL_NTS);
+
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    BOOST_CHECK_EQUAL(GetOdbcErrorState(SQL_HANDLE_STMT, stmt), "42S12");
+}
+
+BOOST_AUTO_TEST_CASE(TestSyntaxError)
+{
+    StartAdditionalNode("Node1");
+
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=PUBLIC");
+
+    SQLCHAR req[] = "INSERT INTO TestType(_key, fasf) VALUES(1, 'some')";
+
+    SQLRETURN ret;
+
+    ret = SQLExecDirect(stmt, req, SQL_NTS);
+
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    BOOST_CHECK_EQUAL(GetOdbcErrorState(SQL_HANDLE_STMT, stmt), "42000");
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc-test/src/parser_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/parser_test.cpp b/modules/platforms/cpp/odbc-test/src/parser_test.cpp
index 7c8a73d..51539ae 100644
--- a/modules/platforms/cpp/odbc-test/src/parser_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/parser_test.cpp
@@ -27,7 +27,7 @@ using namespace ignite::odbc;
 
 struct TestMessage
 {
-    TestMessage()
+    TestMessage() : a(0), b()
     {
         // No-op.
     }
@@ -48,7 +48,7 @@ struct TestMessage
         writer.WriteString(b.data(), static_cast<int32_t>(b.size()));
     }
 
-    void Read(ignite::impl::binary::BinaryReaderImpl& reader)
+    void Read(ignite::impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
     {
         a = reader.ReadInt32();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index e7168fe..4c7e402 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -734,11 +734,25 @@ BOOST_FIXTURE_TEST_SUITE(QueriesTestSuite, QueriesTestSuiteFixture)
 BOOST_AUTO_TEST_CASE(TestLegacyConnection)
 {
     Connect("DRIVER={Apache Ignite};SERVER=127.0.0.1;PORT=11110;SCHEMA=cache");
+
+    InsertTestStrings(10, false);
+    InsertTestBatch(11, 20, 9);
 }
 
 BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_2_1_0)
 {
     Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache;PROTOCOL_VERSION=2.1.0");
+
+    InsertTestStrings(10, false);
+    InsertTestBatch(11, 20, 9);
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_2_1_5)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache;PROTOCOL_VERSION=2.1.5");
+
+    InsertTestStrings(10, false);
+    InsertTestBatch(11, 20, 9);
 }
 
 BOOST_AUTO_TEST_CASE(TestTwoRowsInt8)
@@ -2119,7 +2133,7 @@ BOOST_AUTO_TEST_CASE(TestErrorMessage)
     BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
 
     std::string error = GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt);
-    std::string pattern = "HY000: Table \"B\" not found; SQL statement:\\vSELECT a FROM B.*";
+    std::string pattern = "42000: Table \"B\" not found; SQL statement:\\vSELECT a FROM B.*";
 
     boost::cmatch what;
     if (!boost::regex_match(error.c_str(), what, boost::regex(pattern)))

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am
index 92c85c5..7c64754 100644
--- a/modules/platforms/cpp/odbc/include/Makefile.am
+++ b/modules/platforms/cpp/odbc/include/Makefile.am
@@ -51,6 +51,7 @@ noinst_HEADERS = \
     ignite/odbc/diagnostic/diagnosable.h \
     ignite/odbc/diagnostic/diagnosable_adapter.h \
     ignite/odbc/connection.h \
+    ignite/odbc/odbc_error.h \
     ignite/odbc/message.h \
     ignite/odbc/cursor.h \
     ignite/odbc/common_types.h \

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index 0a3b02f..349147f 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -77,12 +77,42 @@ namespace ignite
                  */
                 S01S02_OPTION_VALUE_CHANGED,
 
+                /** Restricted data type attribute violation. */
+                S07006_RESTRICTION_VIOLATION,
+
                 /** String data, length mismatch. */
                 S22026_DATA_LENGTH_MISMATCH,
 
+                /** Integrity constraint violation. */
+                S23000_INTEGRITY_CONSTRAINT_VIOLATION,
+
                 /** Invalid cursor state. */
                 S24000_INVALID_CURSOR_STATE,
 
+                /** Invalid schema name. */
+                S3F000_INVALID_SCHEMA_NAME,
+
+                /** Syntax error or access violation. */
+                S42000_SYNTAX_ERROR_OR_ACCESS_VIOLATION,
+
+                /** Base table or view already exists. */
+                S42S01_TABLE_OR_VIEW_ALREADY_EXISTS,
+
+                /** Base table or view not found. */
+                S42S02_TABLE_OR_VIEW_NOT_FOUND,
+
+                /** Index already exists. */
+                S42S11_INDEX_ALREADY_EXISTS,
+
+                /** Index not found. */
+                S42S12_INDEX_NOT_FOUND,
+
+                /** Column already exists. */
+                S42S21_COLUMN_ALREADY_EXISTS,
+
+                /** Column not found. */
+                S42S22_COLUMN_NOT_FOUND,
+
                 /** Invalid descriptor index. */
                 S07009_INVALID_DESCRIPTOR_INDEX,
 
@@ -102,6 +132,12 @@ namespace ignite
                 /** The connection specified was not open. */
                 S08003_NOT_CONNECTED,
 
+                /** Server rejected the connection. */
+                S08004_CONNECTION_REJECTED,
+
+                /** Communication link failure. */
+                S08S01_LINK_FAILURE,
+
                 /**
                  * An error occurred for which there was no specific SQLSTATE
                  * and for which no implementation-specific SQLSTATE was defined.
@@ -120,6 +156,9 @@ namespace ignite
                  */
                 SHY003_INVALID_APPLICATION_BUFFER_TYPE,
 
+                /** Invalid SQL data type. */
+                SHY004_INVALID_SQL_DATA_TYPE,
+
                 /** Invalid use of null pointer. */
                 SHY009_INVALID_USE_OF_NULL_POINTER,
 
@@ -155,7 +194,7 @@ namespace ignite
                  * The connection timeout period expired before the data source
                  * responded to the request.
                  */
-                SHYT01_CONNECTIOIN_TIMEOUT,
+                SHYT01_CONNECTION_TIMEOUT,
 
                 /**
                  * Driver does not support this function.
@@ -245,6 +284,88 @@ namespace ignite
             };
         };
 
+        struct ResponseStatus
+        {
+            enum Type
+            {
+                /** Operation completed successfully. */
+                SUCCESS = 0,
+
+                /* 1xxx - parsing errors */
+
+                /** Unknown error, or the one without specific code. */
+                UNKNOWN_ERROR = 1,
+
+                /** General parsing error - for the cases when there's no more specific code available. */
+                PARSING_FAILURE = 1001,
+
+                /** Requested operation is not supported. */
+                UNSUPPORTED_OPERATION = 1002,
+
+                /* 2xxx - analysis errors */
+
+                /** Code encountered SQL statement of some type that it did not expect in current analysis context. */
+                UNEXPECTED_OPERATION = 2001,
+
+                /** Code encountered SQL expression of some type that it did not expect in current analysis context. */
+                UNEXPECTED_ELEMENT_TYPE = 2002,
+
+                /** Analysis detected that the statement is trying to directly UPDATE key or its fields. */
+                KEY_UPDATE = 2003,
+
+                /* 3xxx - database API related runtime errors */
+                /** Required table not found. */
+                TABLE_NOT_FOUND = 3001,
+
+                /** Required table does not have a descriptor set. */
+                NULL_TABLE_DESCRIPTOR = 3002,
+
+                /** Statement type does not match that declared by JDBC driver. */
+                STMT_TYPE_MISMATCH = 3003,
+
+                /** DROP TABLE failed. */
+                TABLE_DROP_FAILED = 3004,
+
+                /** Index already exists. */
+                INDEX_ALREADY_EXISTS = 3005,
+
+                /** Index does not exist. */
+                INDEX_NOT_FOUND = 3006,
+
+                /** Required table already exists. */
+                TABLE_ALREADY_EXISTS = 3007,
+
+                /** Required column not found. */
+                COLUMN_NOT_FOUND = 3008,
+
+                /** Required column already exists. */
+                COLUMN_ALREADY_EXISTS = 3009,
+
+                /** Conversion failure. */
+                CONVERSION_FAILED = 3013,
+
+                /* 4xxx - cache related runtime errors */
+
+                /** Attempt to INSERT a key that is already in cache. */
+                DUPLICATE_KEY = 4001,
+
+                /** Attempt to UPDATE or DELETE a key whose value has been updated concurrently by someone else. */
+                CONCURRENT_UPDATE = 4002,
+
+                /** Attempt to INSERT or MERGE {@code null} key. */
+                NULL_KEY = 4003,
+
+                /** Attempt to INSERT or MERGE {@code null} value. */
+                NULL_VALUE = 4004,
+
+                /** EntryProcessor has thrown an exception during IgniteCache::invokeAll. */
+                ENTRY_PROCESSING = 4005,
+
+                /** Cache not found. */
+                CACHE_NOT_FOUND = 4006
+            };
+        };
+
         /**
          * Convert internal Ignite type into ODBC SQL return code.
          *
@@ -269,7 +390,13 @@ namespace ignite
          */
         EnvironmentAttribute::Type EnvironmentAttributeToInternal(int32_t attr);
 
-
+        /**
+         * Convert request response status to SQL state.
+         *
+         * @param status Response status.
+         * @return SQL state.
+         */
+        SqlState::Type ResponseStatusToSqlState(int32_t status);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h
index 2ec1b5b..46d79ce 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h
@@ -24,6 +24,8 @@ namespace ignite
 {
     namespace odbc
     {
+        class OdbcError;
+
         namespace diagnostic
         {
             /**
@@ -73,6 +75,13 @@ namespace ignite
                  */
                 virtual void AddStatusRecord(SqlState::Type sqlState, const std::string& message) = 0;
 
+                /**
+                 * Add new status record.
+                 *
+                 * @param err Error.
+                 */
+                virtual void AddStatusRecord(const OdbcError& err) = 0;
+
             protected:
                 /**
                  * Default constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h
index 45f47f0..461bfc9 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h
@@ -33,6 +33,7 @@ namespace ignite
 {
     namespace odbc
     {
+        class OdbcError;
         class Connection;
 
         namespace diagnostic
@@ -90,6 +91,13 @@ namespace ignite
                  */
                 virtual void AddStatusRecord(SqlState::Type  sqlState, const std::string& message);
 
+                /**
+                 * Add new status record.
+                 *
+                 * @param err Error.
+                 */
+                virtual void AddStatusRecord(const OdbcError& err);
+
             protected:
                 /**
                  * Constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
index fd7897f..91a808c 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
@@ -64,16 +64,6 @@ namespace ignite
             };
         };
 
-        struct ResponseStatus
-        {
-            enum Type
-            {
-                SUCCESS = 0,
-
-                FAILED = 1
-            };
-        };
-
         /**
          * Handshake request.
          */
@@ -411,14 +401,15 @@ namespace ignite
             /**
              * Read response using provided reader.
              * @param reader Reader.
+             * @param ver Protocol version.
              */
-            void Read(impl::binary::BinaryReaderImpl& reader);
+            void Read(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion& ver);
 
             /**
              * Get request processing status.
              * @return Status.
              */
-            int8_t GetStatus() const
+            int32_t GetStatus() const
             {
                 return status;
             }
@@ -436,11 +427,11 @@ namespace ignite
             /**
              * Read data if response status is ResponseStatus::SUCCESS.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl&);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl&, const ProtocolVersion&);
 
         private:
             /** Request processing status. */
-            int8_t status;
+            int32_t status;
 
             /** Error message. */
             std::string error;
@@ -493,7 +484,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            void Read(impl::binary::BinaryReaderImpl& reader);
+            void Read(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
         private:
             /** Handshake accepted. */
@@ -536,7 +527,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
             /** Query ID. */
             int64_t queryId;
@@ -590,7 +581,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
             /** Query ID. */
             int64_t queryId;
@@ -645,12 +636,22 @@ namespace ignite
                 return errorMessage;
             }
 
+            /**
+             * Get error code.
+             * @return Error code.
+             */
+            int32_t GetErrorCode() const
+            {
+                return errorCode;
+            }
+
         private:
             /**
              * Read response using provided reader.
              * @param reader Reader.
+             * @param ver Protocol version.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion& ver);
 
             /** Affected rows. */
             int64_t affectedRows;
@@ -660,6 +661,9 @@ namespace ignite
 
             /** Error message. */
             std::string errorMessage;
+
+            /** Error code. */
+            int32_t errorCode;
         };
 
         /**
@@ -693,7 +697,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
             /** Query ID. */
             int64_t queryId;
@@ -732,7 +736,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
             /** Columns metadata. */
             meta::ColumnMetaVector meta;
@@ -768,7 +772,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
             /** Columns metadata. */
             meta::TableMetaVector meta;
@@ -804,7 +808,7 @@ namespace ignite
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader);
+            virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&);
 
             /** Columns metadata. */
             std::vector<int8_t> typeIds;

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/ignite/odbc/odbc_error.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/odbc_error.h b/modules/platforms/cpp/odbc/include/ignite/odbc/odbc_error.h
new file mode 100644
index 0000000..361b2b5
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/odbc_error.h
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_ODBC_ERROR
+#define _IGNITE_ODBC_ODBC_ERROR
+
+#include <string>
+
+#include "ignite/odbc/common_types.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        /**
+         * ODBC error.
+         */
+        class OdbcError
+        {
+        public:
+            /**
+             * Constructor.
+             *
+             * @param status SQL status.
+             * @param message Error message.
+             */
+            OdbcError(SqlState::Type status, const std::string& message) :
+                status(status),
+                errMessage(message)
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            OdbcError() :
+                status(SqlState::UNKNOWN),
+                errMessage()
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            ~OdbcError()
+            {
+                // No-op.
+            }
+
+            /**
+             * Get status.
+             * @return Status.
+             */
+            SqlState::Type GetStatus() const
+            {
+                return status;
+            }
+
+            /**
+             * Get error message.
+             * @return Error message.
+             */
+            const std::string& GetErrorMessage() const
+            {
+                return errMessage;
+            }
+
+        private:
+            /** Status. */
+            SqlState::Type status;
+
+            /** Error message. */
+            std::string errMessage;
+        };
+    }
+}
+
+#endif //_IGNITE_ODBC_ODBC_ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
index a91af22..bfea67c 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
@@ -28,6 +28,7 @@
 #include <ignite/impl/binary/binary_reader_impl.h>
 
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/protocol_version.h"
 
 namespace ignite
 {
@@ -44,8 +45,13 @@ namespace ignite
 
             /**
              * Constructor.
+             * @param cap Initial capasity.
              */
-            Parser(int32_t cap = DEFAULT_MEM_ALLOCATION) : inMem(cap), outMem(cap), outStream(&outMem)
+            Parser(int32_t cap = DEFAULT_MEM_ALLOCATION) :
+                protocolVer(ProtocolVersion::GetCurrent()),
+                inMem(cap),
+                outMem(cap),
+                outStream(&outMem)
             {
                 //No-op.
             }
@@ -67,7 +73,7 @@ namespace ignite
             template<typename MsgT>
             void Encode(const MsgT& msg, std::vector<int8_t>& buf)
             {
-                using namespace ignite::impl::binary;
+                using namespace impl::binary;
 
                 ResetState();
 
@@ -90,7 +96,7 @@ namespace ignite
             template<typename MsgT>
             void Decode(MsgT& msg, const std::vector<int8_t>& buf)
             {
-                using namespace ignite::impl::binary;
+                using namespace impl::binary;
 
                 if (inMem.Capacity() < static_cast<int32_t>(buf.size()))
                     inMem.Reallocate(static_cast<int32_t>(buf.size()));
@@ -99,11 +105,21 @@ namespace ignite
 
                 inMem.Length(static_cast<int32_t>(buf.size()));
 
-                ignite::impl::interop::InteropInputStream inStream(&inMem);
+                impl::interop::InteropInputStream inStream(&inMem);
 
                 BinaryReaderImpl reader(&inStream);
 
-                msg.Read(reader);
+                msg.Read(reader, protocolVer);
+            }
+
+            /**
+             * Set protocol version.
+             *
+             * @param ver Version to set.
+             */
+            void SetProtocolVersion(const ProtocolVersion& ver)
+            {
+                protocolVer = ver;
             }
 
         private:
@@ -119,14 +135,17 @@ namespace ignite
                 outStream.Position(0);
             }
 
+            /** Protocol version. */
+            ProtocolVersion protocolVer;
+
             /** Input operational memory. */
-            ignite::impl::interop::InteropUnpooledMemory inMem;
+            impl::interop::InteropUnpooledMemory inMem;
 
             /** Output operational memory. */
-            ignite::impl::interop::InteropUnpooledMemory outMem;
+            impl::interop::InteropUnpooledMemory outMem;
 
             /** Output stream. */
-            ignite::impl::interop::InteropOutputStream outStream;
+            impl::interop::InteropOutputStream outStream;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
index e47f618..c5783ff 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
@@ -217,6 +217,7 @@
     <ClInclude Include="..\..\include\ignite\odbc\meta\column_meta.h" />
     <ClInclude Include="..\..\include\ignite\odbc\meta\primary_key_meta.h" />
     <ClInclude Include="..\..\include\ignite\odbc\meta\table_meta.h" />
+    <ClInclude Include="..\..\include\ignite\odbc\odbc_error.h" />
     <ClInclude Include="..\..\include\ignite\odbc\parser.h" />
     <ClInclude Include="..\..\include\ignite\odbc\protocol_version.h" />
     <ClInclude Include="..\..\include\ignite\odbc\query\batch_query.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
index e252d5d..8934625 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
@@ -281,5 +281,8 @@
     <ClInclude Include="..\..\include\ignite\odbc\query\batch_query.h">
       <Filter>Code\query</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\odbc\odbc_error.h">
+      <Filter>Code</Filter>
+    </ClInclude>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/common_types.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/common_types.cpp b/modules/platforms/cpp/odbc/src/common_types.cpp
index 09ea3dc..10c4f77 100644
--- a/modules/platforms/cpp/odbc/src/common_types.cpp
+++ b/modules/platforms/cpp/odbc/src/common_types.cpp
@@ -118,6 +118,57 @@ namespace ignite
 
             return EnvironmentAttribute::UNKNOWN;
         }
+
+        SqlState::Type ResponseStatusToSqlState(int32_t status)
+        {
+            switch (status)
+            {
+                case ResponseStatus::PARSING_FAILURE:
+                case ResponseStatus::KEY_UPDATE:
+                case ResponseStatus::UNEXPECTED_OPERATION:
+                    return SqlState::S42000_SYNTAX_ERROR_OR_ACCESS_VIOLATION;
+
+                case ResponseStatus::UNSUPPORTED_OPERATION:
+                    return SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED;
+
+                case ResponseStatus::UNEXPECTED_ELEMENT_TYPE:
+                    return SqlState::SHY004_INVALID_SQL_DATA_TYPE;
+
+                case ResponseStatus::DUPLICATE_KEY:
+                case ResponseStatus::NULL_KEY:
+                case ResponseStatus::NULL_VALUE:
+                    return SqlState::S23000_INTEGRITY_CONSTRAINT_VIOLATION;
+
+                case ResponseStatus::TABLE_NOT_FOUND:
+                    return SqlState::S42S02_TABLE_OR_VIEW_NOT_FOUND;
+
+                case ResponseStatus::INDEX_ALREADY_EXISTS:
+                    return SqlState::S42S11_INDEX_ALREADY_EXISTS;
+
+                case ResponseStatus::INDEX_NOT_FOUND:
+                    return SqlState::S42S12_INDEX_NOT_FOUND;
+
+                case ResponseStatus::TABLE_ALREADY_EXISTS:
+                    return SqlState::S42S01_TABLE_OR_VIEW_ALREADY_EXISTS;
+
+                case ResponseStatus::COLUMN_NOT_FOUND:
+                    return SqlState::S42S22_COLUMN_NOT_FOUND;
+
+                case ResponseStatus::COLUMN_ALREADY_EXISTS:
+                    return SqlState::S42S21_COLUMN_ALREADY_EXISTS;
+
+                case ResponseStatus::CACHE_NOT_FOUND:
+                case ResponseStatus::NULL_TABLE_DESCRIPTOR:
+                case ResponseStatus::CONVERSION_FAILED:
+                case ResponseStatus::CONCURRENT_UPDATE:
+                case ResponseStatus::ENTRY_PROCESSING:
+                case ResponseStatus::TABLE_DROP_FAILED:
+                case ResponseStatus::STMT_TYPE_MISMATCH:
+                case ResponseStatus::UNKNOWN_ERROR:
+                default:
+                    return SqlState::SHY000_GENERAL_ERROR;
+            }
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 777973a..161e1c4 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -27,6 +27,7 @@
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/config/configuration.h"
+#include "ignite/odbc/odbc_error.h"
 
 namespace
 {
@@ -138,6 +139,8 @@ namespace ignite
 
             if (res == SqlResult::AI_ERROR)
                 Close();
+            else
+                parser.SetProtocolVersion(config.GetProtocolVersion());
 
             return res;
         }
@@ -194,7 +197,7 @@ namespace ignite
         void Connection::Send(const int8_t* data, size_t len)
         {
             if (!connected)
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_ILLEGAL_STATE, "Connection is not established");
+                throw OdbcError(SqlState::S08003_NOT_CONNECTED, "Connection is not established");
 
             int32_t newLen = static_cast<int32_t>(len + sizeof(OdbcProtocolHeader));
 
@@ -209,7 +212,7 @@ namespace ignite
             size_t sent = SendAll(msg.GetData(), msg.GetSize());
 
             if (sent != len + sizeof(OdbcProtocolHeader))
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message");
+                throw OdbcError(SqlState::S08S01_LINK_FAILURE, "Can not send message due to connection failure");
 
             LOG_MSG("message sent: (" <<  msg.GetSize() << " bytes)" << utility::HexDump(msg.GetData(), msg.GetSize()));
         }
@@ -240,7 +243,7 @@ namespace ignite
         void Connection::Receive(std::vector<int8_t>& msg)
         {
             if (!connected)
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_ILLEGAL_STATE, "Connection is not established");
+                throw OdbcError(SqlState::S08003_NOT_CONNECTED, "Connection is not established");
 
             msg.clear();
 
@@ -249,13 +252,13 @@ namespace ignite
             size_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
 
             if (received != sizeof(hdr))
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message header");
+                throw OdbcError(SqlState::S08S01_LINK_FAILURE, "Can not receive message header");
 
             if (hdr.len < 0)
             {
                 Close();
 
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Message length is negative");
+                throw OdbcError(SqlState::S08S01_LINK_FAILURE, "Protocol error: Message length is negative");
             }
 
             if (hdr.len == 0)
@@ -269,7 +272,7 @@ namespace ignite
             {
                 msg.resize(received);
 
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message body");
+                throw OdbcError(SqlState::S08S01_LINK_FAILURE, "Can not receive message body");
             }
 
             LOG_MSG("Message received: " << utility::HexDump(&msg[0], msg.size()));
@@ -447,9 +450,15 @@ namespace ignite
             {
                 SyncMessage(req, rsp);
             }
+            catch (const OdbcError& err)
+            {
+                AddStatusRecord(err);
+
+                return SqlResult::AI_ERROR;
+            }
             catch (const IgniteError& err)
             {
-                AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                AddStatusRecord(SqlState::S08004_CONNECTION_REJECTED, err.GetText());
 
                 return SqlResult::AI_ERROR;
             }
@@ -469,7 +478,7 @@ namespace ignite
                             << "driver protocol version introduced in version: "
                             << config.GetProtocolVersion().ToString() << ".";
 
-                AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, constructor.str());
+                AddStatusRecord(SqlState::S08004_CONNECTION_REJECTED, constructor.str());
 
                 return SqlResult::AI_ERROR;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
index 6c07c29..096593f 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
@@ -17,6 +17,7 @@
 
 #include "ignite/odbc/log.h"
 #include "ignite/odbc/connection.h"
+#include "ignite/odbc/odbc_error.h"
 #include "ignite/odbc/diagnostic/diagnosable_adapter.h"
 
 namespace ignite
@@ -46,6 +47,13 @@ namespace ignite
 
                 AddStatusRecord(sqlState, message, 0, 0);
             }
+
+            void DiagnosableAdapter::AddStatusRecord(const OdbcError& err)
+            {
+                LOG_MSG("Adding new record: " << err.GetErrorMessage());
+
+                AddStatusRecord(err.GetStatus(), err.GetErrorMessage(), 0, 0);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
index c8e0e81..0a02310 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
@@ -40,15 +40,15 @@ namespace
     /** SQL state 01S01 constant. */
     const std::string STATE_01S01 = "01S01";
 
-    /** SQL state 22026 constant. */
-    const std::string STATE_22026 = "22026";
-
-    /** SQL state 24000 constant. */
-    const std::string STATE_24000 = "24000";
+    /** SQL state 01S02 constant. */
+    const std::string STATE_01S02 = "01S02";
 
     /** SQL state 07009 constant. */
     const std::string STATE_07009 = "07009";
 
+    /** SQL state 07006 constant. */
+    const std::string STATE_07006 = "07006";
+
     /** SQL state 08001 constant. */
     const std::string STATE_08001 = "08001";
 
@@ -58,6 +58,45 @@ namespace
     /** SQL state 08003 constant. */
     const std::string STATE_08003 = "08003";
 
+    /** SQL state 08004 constant. */
+    const std::string STATE_08004 = "08004";
+
+    /** SQL state 08S01 constant. */
+    const std::string STATE_08S01 = "08S01";
+
+    /** SQL state 22026 constant. */
+    const std::string STATE_22026 = "22026";
+
+    /** SQL state 23000 constant. */
+    const std::string STATE_23000 = "23000";
+
+    /** SQL state 24000 constant. */
+    const std::string STATE_24000 = "24000";
+
+    /** SQL state 3F000 constant. */
+    const std::string STATE_3F000 = "3F000";
+
+    /** SQL state 42000 constant. */
+    const std::string STATE_42000 = "42000";
+
+    /** SQL state 42S01 constant. */
+    const std::string STATE_42S01 = "42S01";
+
+    /** SQL state 42S02 constant. */
+    const std::string STATE_42S02 = "42S02";
+
+    /** SQL state 42S11 constant. */
+    const std::string STATE_42S11 = "42S11";
+
+    /** SQL state 42S12 constant. */
+    const std::string STATE_42S12 = "42S12";
+
+    /** SQL state 42S21 constant. */
+    const std::string STATE_42S21 = "42S21";
+
+    /** SQL state 42S22 constant. */
+    const std::string STATE_42S22 = "42S22";
+
     /** SQL state HY000 constant. */
     const std::string STATE_HY000 = "HY000";
 
@@ -67,15 +106,24 @@ namespace
     /** SQL state HY003 constant. */
     const std::string STATE_HY003 = "HY003";
 
+    /** SQL state HY004 constant. */
+    const std::string STATE_HY004 = "HY004";
+
     /** SQL state HY009 constant. */
     const std::string STATE_HY009 = "HY009";
 
     /** SQL state HY010 constant. */
     const std::string STATE_HY010 = "HY010";
 
+    /** SQL state HY090 constant. */
+    const std::string STATE_HY090 = "HY090";
+
     /** SQL state HY092 constant. */
     const std::string STATE_HY092 = "HY092";
 
+    /** SQL state HY097 constant. */
+    const std::string STATE_HY097 = "HY097";
+
     /** SQL state HY105 constant. */
     const std::string STATE_HY105 = "HY105";
 
@@ -88,9 +136,6 @@ namespace
     /** SQL state HYT01 constant. */
     const std::string STATE_HYT01 = "HYT01";
 
-    /** SQL state HY090 constant. */
-    const std::string STATE_HY090 = "HY090";
-
     /** SQL state IM001 constant. */
     const std::string STATE_IM001 = "IM001";
 }
@@ -230,12 +275,45 @@ namespace ignite
                     case SqlState::S01S01_ERROR_IN_ROW:
                         return STATE_01S01;
 
+                    case SqlState::S01S02_OPTION_VALUE_CHANGED:
+                        return STATE_01S02;
+
+                    case SqlState::S07006_RESTRICTION_VIOLATION:
+                        return STATE_07006;
+
                     case SqlState::S22026_DATA_LENGTH_MISMATCH:
                         return STATE_22026;
 
+                    case SqlState::S23000_INTEGRITY_CONSTRAINT_VIOLATION:
+                        return STATE_23000;
+
                     case SqlState::S24000_INVALID_CURSOR_STATE:
                         return STATE_24000;
 
+                    case SqlState::S3F000_INVALID_SCHEMA_NAME:
+                        return STATE_3F000;
+
+                    case SqlState::S42000_SYNTAX_ERROR_OR_ACCESS_VIOLATION:
+                        return STATE_42000;
+
+                    case SqlState::S42S01_TABLE_OR_VIEW_ALREADY_EXISTS:
+                        return STATE_42S01;
+
+                    case SqlState::S42S02_TABLE_OR_VIEW_NOT_FOUND:
+                        return STATE_42S02;
+
+                    case SqlState::S42S11_INDEX_ALREADY_EXISTS:
+                        return STATE_42S11;
+
+                    case SqlState::S42S12_INDEX_NOT_FOUND:
+                        return STATE_42S12;
+
+                    case SqlState::S42S21_COLUMN_ALREADY_EXISTS:
+                        return STATE_42S21;
+
+                    case SqlState::S42S22_COLUMN_NOT_FOUND:
+                        return STATE_42S22;
+
                     case SqlState::S07009_INVALID_DESCRIPTOR_INDEX:
                         return STATE_07009;
 
@@ -248,6 +326,12 @@ namespace ignite
                     case SqlState::S08003_NOT_CONNECTED:
                         return STATE_08003;
 
+                    case SqlState::S08004_CONNECTION_REJECTED:
+                        return STATE_08004;
+
+                    case SqlState::S08S01_LINK_FAILURE:
+                        return STATE_08S01;
+
                     case SqlState::SHY000_GENERAL_ERROR:
                         return STATE_HY000;
 
@@ -269,6 +353,9 @@ namespace ignite
                     case SqlState::SHY092_OPTION_TYPE_OUT_OF_RANGE:
                         return STATE_HY092;
 
+                    case SqlState::SHY097_COLUMN_TYPE_OUT_OF_RANGE:
+                        return STATE_HY097;
+
                     case SqlState::SHY105_INVALID_PARAMETER_TYPE:
                         return STATE_HY105;
 
@@ -278,7 +365,7 @@ namespace ignite
                     case SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED:
                         return STATE_HYC00;
 
-                    case SqlState::SHYT01_CONNECTIOIN_TIMEOUT:
+                    case SqlState::SHYT01_CONNECTION_TIMEOUT:
                         return STATE_HYT01;
 
                     case SqlState::SIM001_FUNCTION_NOT_SUPPORTED:

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/message.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/message.cpp b/modules/platforms/cpp/odbc/src/message.cpp
index 9930290..3601591 100644
--- a/modules/platforms/cpp/odbc/src/message.cpp
+++ b/modules/platforms/cpp/odbc/src/message.cpp
@@ -206,7 +206,9 @@ namespace ignite
             writer.WriteObject<std::string>(sqlQuery);
         }
 
-        Response::Response(): status(ResponseStatus::FAILED), error()
+        Response::Response() :
+            status(ResponseStatus::UNKNOWN_ERROR),
+            error()
         {
             // No-op.
         }
@@ -216,17 +218,20 @@ namespace ignite
             // No-op.
         }
 
-        void Response::Read(impl::binary::BinaryReaderImpl& reader)
+        void Response::Read(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion& ver)
         {
-            status = reader.ReadInt8();
+            if (ver < ProtocolVersion::VERSION_2_1_5)
+                status = reader.ReadInt8();
+            else
+                status = reader.ReadInt32();
 
             if (status == ResponseStatus::SUCCESS)
-                ReadOnSuccess(reader);
+                ReadOnSuccess(reader, ver);
             else
                 utility::ReadString(reader, error);;
         }
 
-        void Response::ReadOnSuccess(impl::binary::BinaryReaderImpl&)
+        void Response::ReadOnSuccess(impl::binary::BinaryReaderImpl&, const ProtocolVersion&)
         {
             // No-op.
         }
@@ -244,7 +249,7 @@ namespace ignite
             // No-op.
         }
 
-        void HandshakeResponse::Read(impl::binary::BinaryReaderImpl& reader)
+        void HandshakeResponse::Read(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             accepted = reader.ReadBool();
 
@@ -270,7 +275,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryCloseResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryCloseResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             queryId = reader.ReadInt64();
         }
@@ -288,7 +293,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryExecuteResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryExecuteResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             queryId = reader.ReadInt64();
 
@@ -300,7 +305,8 @@ namespace ignite
         QueryExecuteBatchResponse::QueryExecuteBatchResponse():
             affectedRows(0),
             errorSetIdx(-1),
-            errorMessage()
+            errorMessage(),
+            errorCode(1)
         {
             // No-op.
         }
@@ -310,7 +316,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryExecuteBatchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryExecuteBatchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion& ver)
         {
             bool success = reader.ReadBool();
             affectedRows = reader.ReadInt64();
@@ -319,6 +325,9 @@ namespace ignite
             {
                 errorSetIdx = reader.ReadInt64();
                 errorMessage = reader.ReadObject<std::string>();
+
+                if (ver >= ProtocolVersion::VERSION_2_1_5)
+                    errorCode = reader.ReadInt32();
             }
         }
 
@@ -332,7 +341,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryFetchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryFetchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             queryId = reader.ReadInt64();
 
@@ -349,7 +358,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryGetColumnsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryGetColumnsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             meta::ReadColumnMetaVector(reader, meta);
         }
@@ -364,7 +373,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryGetTablesMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryGetTablesMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             meta::ReadTableMetaVector(reader, meta);
         }
@@ -379,7 +388,7 @@ namespace ignite
             // No-op.
         }
 
-        void QueryGetParamsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader)
+        void QueryGetParamsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader, const ProtocolVersion&)
         {
             utility::ReadByteArray(reader, typeIds);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/query/batch_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/batch_query.cpp b/modules/platforms/cpp/odbc/src/query/batch_query.cpp
index 46447c0..fc8fda4 100644
--- a/modules/platforms/cpp/odbc/src/query/batch_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/batch_query.cpp
@@ -18,6 +18,7 @@
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/log.h"
+#include "ignite/odbc/odbc_error.h"
 #include "ignite/odbc/query/batch_query.h"
 
 namespace ignite
@@ -160,9 +161,15 @@ namespace ignite
                 {
                     connection.SyncMessage(req, rsp);
                 }
+                catch (const OdbcError& err)
+                {
+                    diag.AddStatusRecord(err);
+
+                    return SqlResult::AI_ERROR;
+                }
                 catch (const IgniteError& err)
                 {
-                    diag.AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -171,7 +178,7 @@ namespace ignite
                 {
                     LOG_MSG("Error: " << rsp.GetError());
 
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -186,7 +193,7 @@ namespace ignite
                     setsProcessed += rsp.GetErrorSetIdx();
                     LOG_MSG("setsProcessed: " << setsProcessed);
 
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetErrorMessage(),
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetErrorCode()), rsp.GetErrorMessage(),
                         static_cast<int32_t>(setsProcessed), 0);
 
                     return SqlResult::AI_SUCCESS_WITH_INFO;

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
index b9c08f5..0910612 100644
--- a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
@@ -21,6 +21,7 @@
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/log.h"
+#include "ignite/odbc/odbc_error.h"
 #include "ignite/odbc/query/column_metadata_query.h"
 
 namespace
@@ -297,9 +298,15 @@ namespace ignite
                 {
                     connection.SyncMessage(req, rsp);
                 }
+                catch (const OdbcError& err)
+                {
+                    diag.AddStatusRecord(err);
+
+                    return SqlResult::AI_ERROR;
+                }
                 catch (const IgniteError& err)
                 {
-                    diag.AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -307,7 +314,7 @@ namespace ignite
                 if (rsp.GetStatus() != ResponseStatus::SUCCESS)
                 {
                     LOG_MSG("Error: " << rsp.GetError());
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                     return SqlResult::AI_ERROR;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/query/data_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp
index 245e0bd..80fcc69 100644
--- a/modules/platforms/cpp/odbc/src/query/data_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp
@@ -18,6 +18,7 @@
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/log.h"
+#include "ignite/odbc/odbc_error.h"
 #include "ignite/odbc/query/data_query.h"
 #include "ignite/odbc/query/batch_query.h"
 
@@ -193,9 +194,15 @@ namespace ignite
                 {
                     connection.SyncMessage(req, rsp);
                 }
+                catch (const OdbcError& err)
+                {
+                    diag.AddStatusRecord(err);
+
+                    return SqlResult::AI_ERROR;
+                }
                 catch (const IgniteError& err)
                 {
-                    diag.AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -204,7 +211,7 @@ namespace ignite
                 {
                     LOG_MSG("Error: " << rsp.GetError());
 
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -241,9 +248,15 @@ namespace ignite
                 {
                     connection.SyncMessage(req, rsp);
                 }
+                catch (const OdbcError& err)
+                {
+                    diag.AddStatusRecord(err);
+
+                    return SqlResult::AI_ERROR;
+                }
                 catch (const IgniteError& err)
                 {
-                    diag.AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -254,7 +267,7 @@ namespace ignite
                 {
                     LOG_MSG("Error: " << rsp.GetError());
 
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -273,9 +286,15 @@ namespace ignite
                 {
                     connection.SyncMessage(req, rsp);
                 }
+                catch (const OdbcError& err)
+                {
+                    diag.AddStatusRecord(err);
+
+                    return SqlResult::AI_ERROR;
+                }
                 catch (const IgniteError& err)
                 {
-                    diag.AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -284,7 +303,7 @@ namespace ignite
                 {
                     LOG_MSG("Error: " << rsp.GetError());
 
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                     return SqlResult::AI_ERROR;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
index e66b281..93f1f79 100644
--- a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
@@ -21,6 +21,7 @@
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/log.h"
+#include "ignite/odbc/odbc_error.h"
 #include "ignite/odbc/query/table_metadata_query.h"
 
 namespace
@@ -223,9 +224,15 @@ namespace ignite
                 {
                     connection.SyncMessage(req, rsp);
                 }
+                catch (const OdbcError& err)
+                {
+                    diag.AddStatusRecord(err);
+
+                    return SqlResult::AI_ERROR;
+                }
                 catch (const IgniteError& err)
                 {
-                    diag.AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                     return SqlResult::AI_ERROR;
                 }
@@ -234,7 +241,7 @@ namespace ignite
                 {
                     LOG_MSG("Error: " << rsp.GetError());
 
-                    diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                    diag.AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                     return SqlResult::AI_ERROR;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc34091c/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index a74a413..36c1a0b 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -29,6 +29,7 @@
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/statement.h"
 #include "ignite/odbc/log.h"
+#include "ignite/odbc/odbc_error.h"
 
 namespace ignite
 {
@@ -1104,9 +1105,15 @@ namespace ignite
             {
                 connection.SyncMessage(req, rsp);
             }
+            catch (const OdbcError& err)
+            {
+                AddStatusRecord(err);
+
+                return SqlResult::AI_ERROR;
+            }
             catch (const IgniteError& err)
             {
-                AddStatusRecord(SqlState::SHYT01_CONNECTIOIN_TIMEOUT, err.GetText());
+                AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText());
 
                 return SqlResult::AI_ERROR;
             }
@@ -1115,7 +1122,7 @@ namespace ignite
             {
                 LOG_MSG("Error: " << rsp.GetError());
 
-                AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, rsp.GetError());
+                AddStatusRecord(ResponseStatusToSqlState(rsp.GetStatus()), rsp.GetError());
 
                 return SqlResult::AI_ERROR;
             }


[22/50] [abbrv] ignite git commit: ignite-6470 Fixed wrong casting og long value to int

Posted by yz...@apache.org.
ignite-6470 Fixed wrong casting og long value to int

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

(cherry picked from commit b98ffab)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: aeb4aeaef44ff7abc152f3942dee9e40086bc69c
Parents: 111d8ab
Author: EdShangGG <es...@gridgain.com>
Authored: Thu Sep 21 18:36:37 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Thu Sep 21 18:55:30 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/persistence/file/FilePageStore.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/aeb4aeae/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 98764a2..0547dbc 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
@@ -538,6 +538,6 @@ public class FilePageStore implements PageStore {
         if (!inited)
             return 0;
 
-        return (int)(allocated.get() - headerSize()) / pageSize;
+        return (int)((allocated.get() - headerSize()) / pageSize);
     }
 }


[29/50] [abbrv] ignite git commit: IGNITE-6476 Fixed "No suitable driver found" in demo mode. (cherry picked from commit 27295f2)

Posted by yz...@apache.org.
IGNITE-6476 Fixed "No suitable driver found" in demo mode.
(cherry picked from commit 27295f2)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: f08127d50357c49999ea24217c096f00942189ed
Parents: cd4e25e
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Sep 22 11:30:12 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Sep 22 11:31:51 2017 +0700

----------------------------------------------------------------------
 .../org/apache/ignite/console/demo/AgentMetadataDemo.java     | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f08127d5/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java
index 4683dd8..b017fb0 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java
@@ -58,6 +58,8 @@ public class AgentMetadataDemo {
             log.info("DEMO: Prepare in-memory H2 database...");
 
             try {
+                Class.forName("org.h2.Driver");
+
                 Connection conn = DriverManager.getConnection("jdbc:h2:mem:demo-db;DB_CLOSE_DELAY=-1", "sa", "");
 
                 File sqlScript = resolvePath("demo/db-init.sql");
@@ -75,6 +77,11 @@ public class AgentMetadataDemo {
 
                 log.info("DEMO: JDBC URL for test drive metadata load: jdbc:h2:mem:demo-db");
             }
+            catch (ClassNotFoundException e) {
+                log.error("DEMO: Failed to load H2 driver!", e);
+
+                throw new SQLException("Failed to load H2 driver", e);
+            }
             catch (SQLException e) {
                 log.error("DEMO: Failed to start test drive for metadata!", e);
 


[02/50] [abbrv] ignite git commit: ignite-6262 Flaky deadlock detection tests are fixed

Posted by yz...@apache.org.
ignite-6262 Flaky deadlock detection tests are fixed

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/4a1e90c6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4a1e90c6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4a1e90c6

Branch: refs/heads/ignite-2.1.5-p1
Commit: 4a1e90c654ce8f169960b6ef5035cdb9fa837a19
Parents: 542f2c2
Author: Vitaliy Biryukov <Bi...@gmail.com>
Authored: Tue Sep 19 13:23:45 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Tue Sep 19 13:31:51 2017 +0300

----------------------------------------------------------------------
 .../AbstractDeadlockDetectionTest.java          |  65 ++++++
 .../TxOptimisticDeadlockDetectionTest.java      |  54 +----
 .../TxPessimisticDeadlockDetectionTest.java     | 223 +++++--------------
 3 files changed, 116 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1e90c6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractDeadlockDetectionTest.java
index 7381f5a..c0034f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/AbstractDeadlockDetectionTest.java
@@ -17,13 +17,25 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
+import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -35,6 +47,59 @@ import java.util.List;
  */
 public abstract class AbstractDeadlockDetectionTest extends GridCommonAbstractTest {
     /**
+     * Checks that transactions and futures are completed and entries are not locked.
+     * @param involvedKeys Involved keys.
+     */
+    protected void checkAllTransactionsCompleted(Set<Object> involvedKeys, int nodesCnt, String cacheName) {
+        boolean fail = false;
+
+        for (int i = 0; i < nodesCnt; i++) {
+            Ignite ignite = ignite(i);
+
+            int cacheId = ((IgniteCacheProxy)ignite.cache(cacheName)).context().cacheId();
+
+            GridCacheSharedContext<Object, Object> cctx = ((IgniteKernal)ignite).context().cache().context();
+
+            IgniteTxManager txMgr = cctx.tm();
+
+            Collection<IgniteInternalTx> activeTxs = txMgr.activeTransactions();
+
+            for (IgniteInternalTx tx : activeTxs) {
+                Collection<IgniteTxEntry> entries = tx.allEntries();
+
+                for (IgniteTxEntry entry : entries) {
+                    if (entry.cacheId() == cacheId) {
+                        fail = true;
+
+                        U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() +
+                            "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx);
+                    }
+                }
+            }
+
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
+
+            assertTrue(futs.isEmpty());
+
+            GridCacheAdapter<Object, Integer> intCache = internalCache(i, cacheName);
+
+            GridCacheConcurrentMap map = intCache.map();
+
+            for (Object key : involvedKeys) {
+                KeyCacheObject keyCacheObj = intCache.context().toCacheKeyObject(key);
+
+                GridCacheMapEntry entry = map.getEntry(intCache.context(), keyCacheObj);
+
+                if (entry != null)
+                    assertNull("Entry still has locks " + entry, entry.mvccAllLocal());
+            }
+        }
+
+        if (fail)
+            fail("Some transactions still exist");
+    }
+
+    /**
      * @param cache Cache.
      * @param cnt Keys count.
      * @param startFrom Start value for keys search.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1e90c6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
index 24e7802..3414227 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.transactions;
 
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -41,12 +40,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -350,53 +344,7 @@ public class TxOptimisticDeadlockDetectionTest extends AbstractDeadlockDetection
 
         assertNotNull("Failed to detect deadlock", deadlockE);
 
-        boolean fail = false;
-
-        // Check transactions, futures and entry locks state.
-        for (int i = 0; i < NODES_CNT * 2; i++) {
-            Ignite ignite = ignite(i);
-
-            int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId();
-
-            GridCacheSharedContext<Object, Object> cctx = ((IgniteKernal)ignite).context().cache().context();
-
-            IgniteTxManager txMgr = cctx.tm();
-
-            Collection<IgniteInternalTx> activeTxs = txMgr.activeTransactions();
-
-            for (IgniteInternalTx tx : activeTxs) {
-                Collection<IgniteTxEntry> entries = tx.allEntries();
-
-                for (IgniteTxEntry entry : entries) {
-                    if (entry.cacheId() == cacheId) {
-                        fail = true;
-
-                        U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() +
-                            "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx);
-                    }
-                }
-            }
-
-            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
-
-            assertTrue(futs.isEmpty());
-
-            GridCacheAdapter<Object, Integer> intCache = internalCache(i, CACHE_NAME);
-
-            GridCacheConcurrentMap map = intCache.map();
-
-            for (Object key : involvedKeys) {
-                KeyCacheObject keyCacheObj = intCache.context().toCacheKeyObject(key);
-
-                GridCacheMapEntry entry = map.getEntry(intCache.context(), keyCacheObj);
-
-                if (entry != null)
-                    assertNull("Entry still has locks " + entry, entry.mvccAllLocal());
-            }
-        }
-
-        if (fail)
-            fail("Some transactions still exist");
+        checkAllTransactionsCompleted(involvedKeys, NODES_CNT * 2, CACHE_NAME);
 
         // Check deadlock report
         String msg = deadlockE.getMessage();

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1e90c6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
index 61f7125..82fa52c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -41,18 +39,11 @@ import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 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.TransactionDeadlockException;
 import org.apache.ignite.transactions.TransactionTimeoutException;
@@ -69,18 +60,18 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 /**
  * Tests deadlock detection for pessimistic transactions.
  */
-public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
+public class TxPessimisticDeadlockDetectionTest extends AbstractDeadlockDetectionTest {
     /** Cache name. */
     private static final String CACHE_NAME = "cache";
 
     /** Nodes count (actually two times more nodes will started: server + client). */
     private static final int NODES_CNT = 4;
 
-    /** No op transformer. */
-    private static final NoOpTransformer NO_OP_TRANSFORMER = new NoOpTransformer();
+    /** Ordinal start key. */
+    private static final Integer ORDINAL_START_KEY = 1;
 
-    /** Wrapping transformer. */
-    private static final WrappingTransformer WRAPPING_TRANSFORMER = new WrappingTransformer();
+    /** Custom start key. */
+    private static final IncrementalTestObject CUSTOM_START_KEY = new KeyObject(1);
 
     /** Client mode flag. */
     private static boolean client;
@@ -141,8 +132,8 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
      */
     public void testDeadlocksPartitioned() throws Exception {
         for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
-            doTestDeadlocks(createCache(PARTITIONED, syncMode, false), NO_OP_TRANSFORMER);
-            doTestDeadlocks(createCache(PARTITIONED, syncMode, false), WRAPPING_TRANSFORMER);
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, false), ORDINAL_START_KEY);
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, false), CUSTOM_START_KEY);
         }
     }
 
@@ -151,8 +142,8 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
      */
     public void testDeadlocksPartitionedNear() throws Exception {
         for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
-            doTestDeadlocks(createCache(PARTITIONED, syncMode, true), NO_OP_TRANSFORMER);
-            doTestDeadlocks(createCache(PARTITIONED, syncMode, true), WRAPPING_TRANSFORMER);
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, true), ORDINAL_START_KEY);
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, true), CUSTOM_START_KEY);
         }
     }
 
@@ -161,8 +152,8 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
      */
     public void testDeadlocksReplicated() throws Exception {
         for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
-            doTestDeadlocks(createCache(REPLICATED, syncMode, false), NO_OP_TRANSFORMER);
-            doTestDeadlocks(createCache(REPLICATED, syncMode, false), WRAPPING_TRANSFORMER);
+            doTestDeadlocks(createCache(REPLICATED, syncMode, false), ORDINAL_START_KEY);
+            doTestDeadlocks(createCache(REPLICATED, syncMode, false), CUSTOM_START_KEY);
         }
     }
 
@@ -178,8 +169,8 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
                 awaitPartitionMapExchange();
 
-                doTestDeadlock(2, true, true, false, NO_OP_TRANSFORMER);
-                doTestDeadlock(2, true, true, false, WRAPPING_TRANSFORMER);
+                doTestDeadlock(2, true, true, false, ORDINAL_START_KEY);
+                doTestDeadlock(2, true, true, false, CUSTOM_START_KEY);
             }
             finally {
                 if (cache != null)
@@ -225,21 +216,21 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    private void doTestDeadlocks(IgniteCache cache, IgniteClosure<Integer, Object> transformer) throws Exception {
+    private void doTestDeadlocks(IgniteCache cache, Object startKey) throws Exception {
         try {
             awaitPartitionMapExchange();
 
-            doTestDeadlock(2, false, true, true, transformer);
-            doTestDeadlock(2, false, false, false, transformer);
-            doTestDeadlock(2, false, false, true, transformer);
+            doTestDeadlock(2, false, true, true, startKey);
+            doTestDeadlock(2, false, false, false, startKey);
+            doTestDeadlock(2, false, false, true, startKey);
 
-            doTestDeadlock(3, false, true, true, transformer);
-            doTestDeadlock(3, false, false, false, transformer);
-            doTestDeadlock(3, false, false, true, transformer);
+            doTestDeadlock(3, false, true, true, startKey);
+            doTestDeadlock(3, false, false, false, startKey);
+            doTestDeadlock(3, false, false, true, startKey);
 
-            doTestDeadlock(4, false, true, true, transformer);
-            doTestDeadlock(4, false, false, false, transformer);
-            doTestDeadlock(4, false, false, true, transformer);
+            doTestDeadlock(4, false, true, true, startKey);
+            doTestDeadlock(4, false, false, false, startKey);
+            doTestDeadlock(4, false, false, true, startKey);
         }
         catch (Exception e) {
             U.error(log, "Unexpected exception: ", e);
@@ -260,10 +251,10 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
         final boolean loc,
         boolean lockPrimaryFirst,
         final boolean clientTx,
-        final IgniteClosure<Integer, Object> transformer
+        final Object startKey
     ) throws Exception {
         log.info(">>> Test deadlock [txCnt=" + txCnt + ", loc=" + loc + ", lockPrimaryFirst=" + lockPrimaryFirst +
-            ", clientTx=" + clientTx + ", transformer=" + transformer.getClass().getName() + ']');
+            ", clientTx=" + clientTx + ", startKey=" + startKey.getClass().getName() + ']');
 
         final AtomicInteger threadCnt = new AtomicInteger();
 
@@ -271,10 +262,10 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         final AtomicReference<TransactionDeadlockException> deadlockErr = new AtomicReference<>();
 
-        final List<List<Integer>> keySets = generateKeys(txCnt, loc, !lockPrimaryFirst);
+        final List<List<Object>> keySets = generateKeys(txCnt, startKey, loc, !lockPrimaryFirst);
 
-        final Set<Integer> involvedKeys = new GridConcurrentHashSet<>();
-        final Set<Integer> involvedLockedKeys = new GridConcurrentHashSet<>();
+        final Set<Object> involvedKeys = new GridConcurrentHashSet<>();
+        final Set<Object> involvedLockedKeys = new GridConcurrentHashSet<>();
         final Set<IgniteInternalTx> involvedTxs = new GridConcurrentHashSet<>();
 
         IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
@@ -285,23 +276,23 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
                 IgniteCache<Object, Integer> cache = ignite.cache(CACHE_NAME);
 
-                List<Integer> keys = keySets.get(threadNum - 1);
+                List<Object> keys = keySets.get(threadNum - 1);
 
                 int txTimeout = 500 + txCnt * 100;
 
                 try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, txTimeout, 0)) {
                     involvedTxs.add(((TransactionProxyImpl)tx).tx());
 
-                    Integer key = keys.get(0);
+                    Object key = keys.get(0);
 
                     involvedKeys.add(key);
 
                     Object k;
 
                     log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + transformer.apply(key) + ']');
+                        ", tx=" + tx + ", key=" + key + ']');
 
-                    cache.put(transformer.apply(key), 0);
+                    cache.put(key, 0);
 
                     involvedLockedKeys.add(key);
 
@@ -312,23 +303,23 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
                     ClusterNode primaryNode =
                         ((IgniteCacheProxy)cache).context().affinity().primaryByKey(key, NONE);
 
-                    List<Integer> primaryKeys =
-                        primaryKeys(grid(primaryNode).cache(CACHE_NAME), 5, key + (100 * threadNum));
+                    List<Object> primaryKeys =
+                        primaryKeys(grid(primaryNode).cache(CACHE_NAME), 5, incrementKey(key, 100 * threadNum));
 
                     Map<Object, Integer> entries = new HashMap<>();
 
                     involvedKeys.add(key);
 
-                    entries.put(transformer.apply(key), 0);
+                    entries.put(key, 0);
 
-                    for (Integer i : primaryKeys) {
-                        involvedKeys.add(i);
+                    for (Object o : primaryKeys) {
+                        involvedKeys.add(o);
 
-                        entries.put(transformer.apply(i), 1);
+                        entries.put(o, 1);
 
-                        k = transformer.apply(i + 13);
+                        k = incrementKey(o, + 13);
 
-                        involvedKeys.add(i + 13);
+                        involvedKeys.add(k);
 
                         entries.put(k, 2);
                     }
@@ -368,55 +359,7 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         assertNotNull(deadlockE);
 
-        boolean fail = false;
-
-        // Check transactions, futures and entry locks state.
-        for (int i = 0; i < NODES_CNT * 2; i++) {
-            Ignite ignite = ignite(i);
-
-            int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId();
-
-            GridCacheSharedContext<Object, Object> cctx = ((IgniteKernal)ignite).context().cache().context();
-
-            IgniteTxManager txMgr = cctx.tm();
-
-            Collection<IgniteInternalTx> activeTxs = txMgr.activeTransactions();
-
-            for (IgniteInternalTx tx : activeTxs) {
-                Collection<IgniteTxEntry> entries = tx.allEntries();
-
-                for (IgniteTxEntry entry : entries) {
-                    if (entry.cacheId() == cacheId) {
-                        fail = true;
-
-                        U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() +
-                            "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx);
-                    }
-                }
-            }
-
-            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
-
-            assertTrue(futs.isEmpty());
-
-            GridCacheAdapter<Object, Integer> intCache = internalCache(i, CACHE_NAME);
-
-            GridCacheConcurrentMap map = intCache.map();
-
-            for (Integer key : involvedKeys) {
-                Object key0 = transformer.apply(key);
-
-                KeyCacheObject keyCacheObj = intCache.context().toCacheKeyObject(key0);
-
-                GridCacheMapEntry entry = map.getEntry(intCache.context(), keyCacheObj);
-
-                if (entry != null)
-                    assertNull("Entry still has locks " + entry, entry.mvccAllLocal());
-            }
-        }
-
-        if (fail)
-            fail("Some transactions still exist");
+        checkAllTransactionsCompleted(involvedKeys, NODES_CNT * 2, CACHE_NAME);
 
         // Check deadlock report
         String msg = deadlockE.getMessage();
@@ -425,11 +368,11 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             assertTrue(msg.contains(
                 "[txId=" + tx.xidVersion() + ", nodeId=" + tx.nodeId() + ", threadId=" + tx.threadId() + ']'));
 
-        for (Integer key : involvedKeys) {
+        for (Object key : involvedKeys) {
             if (involvedLockedKeys.contains(key))
-                assertTrue(msg.contains("[key=" + transformer.apply(key) + ", cache=" + CACHE_NAME + ']'));
+                assertTrue(msg.contains("[key=" + key + ", cache=" + CACHE_NAME + ']'));
             else
-                assertFalse(msg.contains("[key=" + transformer.apply(key)));
+                assertFalse(msg.contains("[key=" + key));
         }
     }
 
@@ -437,11 +380,11 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
      * @param nodesCnt Nodes count.
      * @param loc Local cache.
      */
-    private List<List<Integer>> generateKeys(int nodesCnt, boolean loc, boolean reverse) throws IgniteCheckedException {
-        List<List<Integer>> keySets = new ArrayList<>();
+    private <T> List<List<T>> generateKeys(int nodesCnt, T startKey, boolean loc, boolean reverse) throws IgniteCheckedException {
+        List<List<T>> keySets = new ArrayList<>();
 
         if (loc) {
-            List<Integer> keys = primaryKeys(ignite(0).cache(CACHE_NAME), 2);
+            List<T> keys = primaryKeys(ignite(0).cache(CACHE_NAME), 2, startKey);
 
             keySets.add(new ArrayList<>(keys));
 
@@ -451,10 +394,10 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
         }
         else {
             for (int i = 0; i < nodesCnt; i++) {
-                List<Integer> keys = new ArrayList<>(2);
+                List<T> keys = new ArrayList<>(2);
 
-                keys.add(primaryKey(ignite(i).cache(CACHE_NAME)));
-                keys.add(primaryKey(ignite(i == nodesCnt - 1 ? 0 : i + 1).cache(CACHE_NAME)));
+                keys.add(primaryKey(ignite(i).cache(CACHE_NAME), startKey));
+                keys.add(primaryKey(ignite(i == nodesCnt - 1 ? 0 : i + 1).cache(CACHE_NAME), startKey));
 
                 if (reverse)
                     Collections.reverse(keys);
@@ -465,70 +408,4 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         return keySets;
     }
-
-    /**
-     *
-     */
-    private static class NoOpTransformer implements IgniteClosure<Integer, Object> {
-        /** {@inheritDoc} */
-        @Override public Object apply(Integer val) {
-            return val;
-        }
-    }
-
-    /**
-     *
-     */
-    private static class WrappingTransformer implements IgniteClosure<Integer, Object> {
-        /** {@inheritDoc} */
-        @Override public Object apply(Integer val) {
-            return new KeyObject(val);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class KeyObject implements Serializable {
-        /** Id. */
-        private int id;
-
-        /** Name. */
-        private String name;
-
-        /**
-         * @param id Id.
-         */
-        public KeyObject(int id) {
-            this.id = id;
-            this.name = "KeyObject" + id;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "KeyObject{" +
-                "id=" + id +
-                ", name='" + name + '\'' +
-                '}';
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            KeyObject obj = (KeyObject)o;
-
-            return id == obj.id && name.equals(obj.name);
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return id;
-        }
-    }
 }


[44/50] [abbrv] ignite git commit: IGNITE-6434 Fixed error in checkpointer during topology change. Fixes #2718

Posted by yz...@apache.org.
IGNITE-6434 Fixed error in checkpointer during topology change. Fixes #2718


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 21de1c56268c18685dd3620b7e3dc776ca2cf532
Parents: ae9c6d6
Author: Eduard Shangareev <ed...@gmail.com>
Authored: Fri Sep 22 16:17:42 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Sep 22 16:18:36 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 495 ++++++++++---------
 .../GridCacheDatabaseSharedManager.java         |   5 +-
 .../IgnitePdsExchangeDuringCheckpointTest.java  | 135 +++++
 .../IgniteCacheDataStructuresSelfTestSuite.java |   4 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   4 +
 5 files changed, 407 insertions(+), 236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/21de1c56/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index cad21d3..5a1e050 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -298,20 +298,27 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         GridDhtPartitionsExchangeFuture exchFut)
         throws IgniteInterruptedCheckedException
     {
-        U.writeLock(lock);
+        ctx.database().checkpointReadLock();
 
         try {
-            if (stopping)
-                return;
+            U.writeLock(lock);
 
-            long updateSeq = this.updateSeq.incrementAndGet();
+            try {
+                if (stopping)
+                    return;
 
-            initPartitions0(affVer, exchFut, updateSeq);
+                long updateSeq = this.updateSeq.incrementAndGet();
 
-            consistencyCheck();
+                initPartitions0(affVer, exchFut, updateSeq);
+
+                consistencyCheck();
+            }
+            finally {
+                lock.writeLock().unlock();
+            }
         }
         finally {
-            lock.writeLock().unlock();
+            ctx.database().checkpointReadUnlock();
         }
     }
 
@@ -589,101 +596,109 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             ", affVer=" + grp.affinity().lastVersion() +
             ", fut=" + exchFut + ']';
 
-        lock.writeLock().lock();
+        ctx.database().checkpointReadLock();
 
         try {
-            if (stopping)
-                return false;
 
-            assert readyTopVer.initialized() : readyTopVer;
-            assert lastTopChangeVer.equals(readyTopVer);
+            lock.writeLock().lock();
 
-            if (log.isDebugEnabled())
-                log.debug("Partition map before afterExchange [exchId=" + exchFut.exchangeId() + ", fullMap=" +
-                    fullMapString() + ']');
+            try {
+                if (stopping)
+                    return false;
 
-            long updateSeq = this.updateSeq.incrementAndGet();
+                assert readyTopVer.initialized() : readyTopVer;
+                assert lastTopChangeVer.equals(readyTopVer);
 
-            for (int p = 0; p < num; p++) {
-                GridDhtLocalPartition locPart = localPartition0(p, topVer, false, false, false);
+                if (log.isDebugEnabled())
+                    log.debug("Partition map before afterExchange [exchId=" + exchFut.exchangeId() + ", fullMap=" +
+                        fullMapString() + ']');
 
-                if (partitionLocalNode(p, topVer)) {
-                    // This partition will be created during next topology event,
-                    // which obviously has not happened at this point.
-                    if (locPart == null) {
-                        if (log.isDebugEnabled())
-                            log.debug("Skipping local partition afterExchange (will not create): " + p);
+                long updateSeq = this.updateSeq.incrementAndGet();
 
-                        continue;
-                    }
+                for (int p = 0; p < num; p++) {
+                    GridDhtLocalPartition locPart = localPartition0(p, topVer, false, false, false);
 
-                    GridDhtPartitionState state = locPart.state();
+                    if (partitionLocalNode(p, topVer)) {
+                        // This partition will be created during next topology event,
+                        // which obviously has not happened at this point.
+                        if (locPart == null) {
+                            if (log.isDebugEnabled())
+                                log.debug("Skipping local partition afterExchange (will not create): " + p);
 
-                    if (state == MOVING) {
-                        if (grp.rebalanceEnabled()) {
-                            Collection<ClusterNode> owners = owners(p);
+                            continue;
+                        }
 
-                            // If there are no other owners, then become an owner.
-                            if (F.isEmpty(owners)) {
-                                boolean owned = locPart.own();
+                        GridDhtPartitionState state = locPart.state();
 
-                                assert owned : "Failed to own partition [grp=" + grp.cacheOrGroupName() + ", locPart=" +
-                                    locPart + ']';
+                        if (state == MOVING) {
+                            if (grp.rebalanceEnabled()) {
+                                Collection<ClusterNode> owners = owners(p);
 
-                                updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
+                                // If there are no other owners, then become an owner.
+                                if (F.isEmpty(owners)) {
+                                    boolean owned = locPart.own();
 
-                                changed = true;
+                                    assert owned : "Failed to own partition [grp=" + grp.cacheOrGroupName() + ", locPart=" +
+                                        locPart + ']';
 
-                                if (grp.eventRecordable(EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
-                                    DiscoveryEvent discoEvt = exchFut.events().lastEvent();
+                                    updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
 
-                                    grp.addRebalanceEvent(p,
-                                        EVT_CACHE_REBALANCE_PART_DATA_LOST,
-                                        discoEvt.eventNode(),
-                                        discoEvt.type(),
-                                        discoEvt.timestamp());
-                                }
+                                    changed = true;
 
-                                if (log.isDebugEnabled())
-                                    log.debug("Owned partition: " + locPart);
+                                    if (grp.eventRecordable(EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
+                                        DiscoveryEvent discoEvt = exchFut.events().lastEvent();
+
+                                        grp.addRebalanceEvent(p,
+                                            EVT_CACHE_REBALANCE_PART_DATA_LOST,
+                                            discoEvt.eventNode(),
+                                            discoEvt.type(),
+                                            discoEvt.timestamp());
+                                    }
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Owned partition: " + locPart);
+                                }
+                                else if (log.isDebugEnabled())
+                                    log.debug("Will not own partition (there are owners to rebalance from) [locPart=" +
+                                        locPart + ", owners = " + owners + ']');
                             }
-                            else if (log.isDebugEnabled())
-                                log.debug("Will not own partition (there are owners to rebalance from) [locPart=" +
-                                    locPart + ", owners = " + owners + ']');
+                            else
+                                updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
                         }
-                        else
-                            updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
                     }
-                }
-                else {
-                    if (locPart != null) {
-                        GridDhtPartitionState state = locPart.state();
+                    else {
+                        if (locPart != null) {
+                            GridDhtPartitionState state = locPart.state();
 
-                        if (state == MOVING) {
-                            locPart.rent(false);
+                            if (state == MOVING) {
+                                locPart.rent(false);
 
-                            updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
+                                updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
 
-                            changed = true;
+                                changed = true;
 
-                            if (log.isDebugEnabled())
-                                log.debug("Evicting moving partition (it does not belong to affinity): " + locPart);
+                                if (log.isDebugEnabled())
+                                    log.debug("Evicting moving partition (it does not belong to affinity): " + locPart);
+                            }
                         }
                     }
                 }
-            }
 
-            AffinityAssignment aff = grp.affinity().readyAffinity(topVer);
+                AffinityAssignment aff = grp.affinity().readyAffinity(topVer);
 
-            if (node2part != null && node2part.valid())
-                changed |= checkEvictions(updateSeq, aff);
+                if (node2part != null && node2part.valid())
+                    changed |= checkEvictions(updateSeq, aff);
 
-            updateRebalanceVersion(aff.assignment());
+                updateRebalanceVersion(aff.assignment());
 
-            consistencyCheck();
+                consistencyCheck();
+            }
+            finally {
+                lock.writeLock().unlock();
+            }
         }
         finally {
-            lock.writeLock().unlock();
+            ctx.database().checkpointReadUnlock();
         }
 
         return changed;
@@ -709,6 +724,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     private GridDhtLocalPartition createPartition(int p) {
         assert lock.isWriteLockedByCurrentThread();
 
+        assert ctx.database().checkpointLockIsHeldByThread();
+
         GridDhtLocalPartition loc = locParts.get(p);
 
         if (loc == null || loc.state() == EVICTED) {
@@ -1183,232 +1200,239 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
         assert partMap != null;
 
-        lock.writeLock().lock();
+        ctx.database().checkpointReadLock();
 
         try {
-            if (stopping || !lastTopChangeVer.initialized() ||
-                // Ignore message not-related to exchange if exchange is in progress.
-                (exchangeVer == null && !lastTopChangeVer.equals(readyTopVer)))
-                return false;
+            lock.writeLock().lock();
 
-            if (incomeCntrMap != null) {
-                // update local counters in partitions
-                for (int i = 0; i < locParts.length(); i++) {
-                    GridDhtLocalPartition part = locParts.get(i);
+            try {
+                if (stopping || !lastTopChangeVer.initialized() ||
+                    // Ignore message not-related to exchange if exchange is in progress.
+                    (exchangeVer == null && !lastTopChangeVer.equals(readyTopVer)))
+                    return false;
 
-                    if (part == null)
-                        continue;
+                if (incomeCntrMap != null) {
+                    // update local counters in partitions
+                    for (int i = 0; i < locParts.length(); i++) {
+                        GridDhtLocalPartition part = locParts.get(i);
+
+                        if (part == null)
+                            continue;
 
-                    if (part.state() == OWNING || part.state() == MOVING) {
-                        long updCntr = incomeCntrMap.updateCounter(part.id());
+                        if (part.state() == OWNING || part.state() == MOVING) {
+                            long updCntr = incomeCntrMap.updateCounter(part.id());
 
-                        if (updCntr != 0 && updCntr > part.updateCounter())
-                            part.updateCounter(updCntr);
+                            if (updCntr != 0 && updCntr > part.updateCounter())
+                                part.updateCounter(updCntr);
+                        }
                     }
                 }
-            }
 
-            if (exchangeVer != null) {
-                // Ignore if exchange already finished or new exchange started.
-                if (readyTopVer.compareTo(exchangeVer) > 0 || lastTopChangeVer.compareTo(exchangeVer) > 0) {
-                    U.warn(log, "Stale exchange id for full partition map update (will ignore) [" +
+                if (exchangeVer != null) {
+                    // Ignore if exchange already finished or new exchange started.
+                    if (readyTopVer.compareTo(exchangeVer) > 0 || lastTopChangeVer.compareTo(exchangeVer) > 0) {
+                        U.warn(log, "Stale exchange id for full partition map update (will ignore) [" +
+                            "lastTopChange=" + lastTopChangeVer +
+                            ", readTopVer=" + readyTopVer +
+                            ", exchVer=" + exchangeVer + ']');
+
+                        return false;
+                    }
+                }
+
+                if (msgTopVer != null && lastTopChangeVer.compareTo(msgTopVer) > 0) {
+                    U.warn(log, "Stale version for full partition map update message (will ignore) [" +
                         "lastTopChange=" + lastTopChangeVer +
                         ", readTopVer=" + readyTopVer +
-                        ", exchVer=" + exchangeVer + ']');
+                        ", msgVer=" + msgTopVer + ']');
 
                     return false;
                 }
-            }
-
-            if (msgTopVer != null && lastTopChangeVer.compareTo(msgTopVer) > 0) {
-                U.warn(log, "Stale version for full partition map update message (will ignore) [" +
-                    "lastTopChange=" + lastTopChangeVer +
-                    ", readTopVer=" + readyTopVer +
-                    ", msgVer=" + msgTopVer + ']');
 
-                return false;
-            }
+                boolean fullMapUpdated = (node2part == null);
 
-            boolean fullMapUpdated = (node2part == null);
+                if (node2part != null) {
+                    for (GridDhtPartitionMap part : node2part.values()) {
+                        GridDhtPartitionMap newPart = partMap.get(part.nodeId());
 
-            if (node2part != null) {
-                for (GridDhtPartitionMap part : node2part.values()) {
-                    GridDhtPartitionMap newPart = partMap.get(part.nodeId());
+                        if (shouldOverridePartitionMap(part, newPart)) {
+                            fullMapUpdated = true;
 
-                    if (shouldOverridePartitionMap(part, newPart)) {
-                        fullMapUpdated = true;
+                            if (log.isDebugEnabled()) {
+                                log.debug("Overriding partition map in full update map [exchVer=" + exchangeVer +
+                                    ", curPart=" + mapString(part) +
+                                    ", newPart=" + mapString(newPart) + ']');
+                            }
 
-                        if (log.isDebugEnabled()) {
-                            log.debug("Overriding partition map in full update map [exchVer=" + exchangeVer +
-                                ", curPart=" + mapString(part) +
-                                ", newPart=" + mapString(newPart) + ']');
+                            if (newPart.nodeId().equals(ctx.localNodeId()))
+                                updateSeq.setIfGreater(newPart.updateSequence());
+                        }
+                        else {
+                            // If for some nodes current partition has a newer map,
+                            // then we keep the newer value.
+                            partMap.put(part.nodeId(), part);
                         }
-
-                        if (newPart.nodeId().equals(ctx.localNodeId()))
-                            updateSeq.setIfGreater(newPart.updateSequence());
-                    }
-                    else {
-                        // If for some nodes current partition has a newer map,
-                        // then we keep the newer value.
-                        partMap.put(part.nodeId(), part);
                     }
-                }
 
-                // Check that we have new nodes.
-                for (GridDhtPartitionMap part : partMap.values()) {
-                    if (fullMapUpdated)
-                        break;
+                    // Check that we have new nodes.
+                    for (GridDhtPartitionMap part : partMap.values()) {
+                        if (fullMapUpdated)
+                            break;
 
-                    fullMapUpdated = !node2part.containsKey(part.nodeId());
-                }
+                        fullMapUpdated = !node2part.containsKey(part.nodeId());
+                    }
 
-                // Remove entry if node left.
-                for (Iterator<UUID> it = partMap.keySet().iterator(); it.hasNext(); ) {
-                    UUID nodeId = it.next();
+                    // Remove entry if node left.
+                    for (Iterator<UUID> it = partMap.keySet().iterator(); it.hasNext(); ) {
+                        UUID nodeId = it.next();
 
-                    if (!ctx.discovery().alive(nodeId)) {
-                        if (log.isDebugEnabled())
-                            log.debug("Removing left node from full map update [nodeId=" + nodeId + ", partMap=" +
-                                partMap + ']');
+                        if (!ctx.discovery().alive(nodeId)) {
+                            if (log.isDebugEnabled())
+                                log.debug("Removing left node from full map update [nodeId=" + nodeId + ", partMap=" +
+                                    partMap + ']');
 
-                        it.remove();
+                            it.remove();
+                        }
                     }
                 }
-            }
-            else {
-                GridDhtPartitionMap locNodeMap = partMap.get(ctx.localNodeId());
-
-                if (locNodeMap != null)
-                    updateSeq.setIfGreater(locNodeMap.updateSequence());
-            }
+                else {
+                    GridDhtPartitionMap locNodeMap = partMap.get(ctx.localNodeId());
 
-            if (!fullMapUpdated) {
-                if (log.isDebugEnabled()) {
-                    log.debug("No updates for full partition map (will ignore) [lastExch=" + lastTopChangeVer +
-                        ", exchVer=" + exchangeVer +
-                        ", curMap=" + node2part +
-                        ", newMap=" + partMap + ']');
+                    if (locNodeMap != null)
+                        updateSeq.setIfGreater(locNodeMap.updateSequence());
                 }
 
-                return false;
-            }
+                if (!fullMapUpdated) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("No updates for full partition map (will ignore) [lastExch=" + lastTopChangeVer +
+                            ", exchVer=" + exchangeVer +
+                            ", curMap=" + node2part +
+                            ", newMap=" + partMap + ']');
+                    }
 
-            if (exchangeVer != null) {
-                assert exchangeVer.compareTo(readyTopVer) >= 0 && exchangeVer.compareTo(lastTopChangeVer) >= 0;
+                    return false;
+                }
 
-                lastTopChangeVer = readyTopVer = exchangeVer;
-            }
+                if (exchangeVer != null) {
+                    assert exchangeVer.compareTo(readyTopVer) >= 0 && exchangeVer.compareTo(lastTopChangeVer) >= 0;
 
-            node2part = partMap;
+                    lastTopChangeVer = readyTopVer = exchangeVer;
+                }
 
-            if (exchangeVer == null && !grp.isReplicated() &&
-                    (readyTopVer.initialized() && readyTopVer.compareTo(diffFromAffinityVer) >= 0)) {
-                AffinityAssignment affAssignment = grp.affinity().readyAffinity(readyTopVer);
+                node2part = partMap;
 
-                for (Map.Entry<UUID, GridDhtPartitionMap> e : partMap.entrySet()) {
-                    for (Map.Entry<Integer, GridDhtPartitionState> e0 : e.getValue().entrySet()) {
-                        int p = e0.getKey();
+                if (exchangeVer == null && !grp.isReplicated() &&
+                        (readyTopVer.initialized() && readyTopVer.compareTo(diffFromAffinityVer) >= 0)) {
+                    AffinityAssignment affAssignment = grp.affinity().readyAffinity(readyTopVer);
 
-                        Set<UUID> diffIds = diffFromAffinity.get(p);
+                    for (Map.Entry<UUID, GridDhtPartitionMap> e : partMap.entrySet()) {
+                        for (Map.Entry<Integer, GridDhtPartitionState> e0 : e.getValue().entrySet()) {
+                            int p = e0.getKey();
 
-                        if ((e0.getValue() == MOVING || e0.getValue() == OWNING || e0.getValue() == RENTING) &&
-                            !affAssignment.getIds(p).contains(e.getKey())) {
+                            Set<UUID> diffIds = diffFromAffinity.get(p);
 
-                            if (diffIds == null)
-                                diffFromAffinity.put(p, diffIds = U.newHashSet(3));
+                            if ((e0.getValue() == MOVING || e0.getValue() == OWNING || e0.getValue() == RENTING) &&
+                                !affAssignment.getIds(p).contains(e.getKey())) {
 
-                            diffIds.add(e.getKey());
-                        }
-                        else {
-                            if (diffIds != null && diffIds.remove(e.getKey())) {
-                                if (diffIds.isEmpty())
-                                    diffFromAffinity.remove(p);
+                                if (diffIds == null)
+                                    diffFromAffinity.put(p, diffIds = U.newHashSet(3));
+
+                                diffIds.add(e.getKey());
+                            }
+                            else {
+                                if (diffIds != null && diffIds.remove(e.getKey())) {
+                                    if (diffIds.isEmpty())
+                                        diffFromAffinity.remove(p);
+                                }
                             }
                         }
                     }
-                }
 
-                diffFromAffinityVer = readyTopVer;
-            }
+                    diffFromAffinityVer = readyTopVer;
+                }
 
-            boolean changed = false;
+                boolean changed = false;
 
-            GridDhtPartitionMap nodeMap = partMap.get(ctx.localNodeId());
+                GridDhtPartitionMap nodeMap = partMap.get(ctx.localNodeId());
 
-            if (nodeMap != null && ctx.database().persistenceEnabled() && readyTopVer.initialized()) {
-                for (Map.Entry<Integer, GridDhtPartitionState> e : nodeMap.entrySet()) {
-                    int p = e.getKey();
-                    GridDhtPartitionState state = e.getValue();
+                if (nodeMap != null && ctx.database().persistenceEnabled() && readyTopVer.initialized()) {
+                    for (Map.Entry<Integer, GridDhtPartitionState> e : nodeMap.entrySet()) {
+                        int p = e.getKey();
+                        GridDhtPartitionState state = e.getValue();
 
-                    if (state == OWNING) {
-                        GridDhtLocalPartition locPart = locParts.get(p);
+                        if (state == OWNING) {
+                            GridDhtLocalPartition locPart = locParts.get(p);
 
-                        assert locPart != null : grp.cacheOrGroupName();
+                            assert locPart != null : grp.cacheOrGroupName();
 
-                        if (locPart.state() == MOVING) {
-                            boolean success = locPart.own();
+                            if (locPart.state() == MOVING) {
+                                boolean success = locPart.own();
 
-                            assert success : locPart;
+                                assert success : locPart;
 
-                            changed |= success;
+                                changed |= success;
+                            }
                         }
-                    }
-                    else if (state == MOVING) {
-                        GridDhtLocalPartition locPart = locParts.get(p);
+                        else if (state == MOVING) {
+                            GridDhtLocalPartition locPart = locParts.get(p);
 
-                        if (locPart == null || locPart.state() == EVICTED)
-                            locPart = createPartition(p);
+                            if (locPart == null || locPart.state() == EVICTED)
+                                locPart = createPartition(p);
 
-                        if (locPart.state() == OWNING) {
-                            locPart.moving();
+                            if (locPart.state() == OWNING) {
+                                locPart.moving();
 
-                            changed = true;
+                                changed = true;
+                            }
                         }
-                    }
-                    else if (state == RENTING && partsToReload.contains(p)) {
-                        GridDhtLocalPartition locPart = locParts.get(p);
+                        else if (state == RENTING && partsToReload.contains(p)) {
+                            GridDhtLocalPartition locPart = locParts.get(p);
 
-                        if (locPart == null || locPart.state() == EVICTED) {
-                            createPartition(p);
+                            if (locPart == null || locPart.state() == EVICTED) {
+                                createPartition(p);
 
-                            changed = true;
-                        }
-                        else if (locPart.state() == OWNING || locPart.state() == MOVING) {
-                            locPart.reload(true);
+                                changed = true;
+                            }
+                            else if (locPart.state() == OWNING || locPart.state() == MOVING) {
+                                locPart.reload(true);
 
-                            locPart.rent(false);
+                                locPart.rent(false);
 
-                            changed = true;
+                                changed = true;
+                            }
+                            else
+                                locPart.reload(true);
                         }
-                        else
-                            locPart.reload(true);
                     }
                 }
-            }
 
-            long updateSeq = this.updateSeq.incrementAndGet();
+                long updateSeq = this.updateSeq.incrementAndGet();
 
-            if (readyTopVer.initialized() && readyTopVer.equals(lastTopChangeVer)) {
-                AffinityAssignment  aff = grp.affinity().readyAffinity(readyTopVer);
+                if (readyTopVer.initialized() && readyTopVer.equals(lastTopChangeVer)) {
+                    AffinityAssignment aff = grp.affinity().readyAffinity(readyTopVer);
 
-                if (exchangeVer == null)
-                    changed |= checkEvictions(updateSeq, aff);
+                    if (exchangeVer == null)
+                        changed |= checkEvictions(updateSeq, aff);
 
-                updateRebalanceVersion(aff.assignment());
-            }
+                    updateRebalanceVersion(aff.assignment());
+                }
 
-            consistencyCheck();
+                consistencyCheck();
 
-            if (log.isDebugEnabled())
-                log.debug("Partition map after full update: " + fullMapString());
+                if (log.isDebugEnabled())
+                    log.debug("Partition map after full update: " + fullMapString());
 
-            if (changed)
-                ctx.exchange().scheduleResendPartitions();
+                if (changed)
+                    ctx.exchange().scheduleResendPartitions();
 
-            return changed;
+                return changed;
+            }
+            finally {
+                lock.writeLock().unlock();
+            }
         }
         finally {
-            lock.writeLock().unlock();
+            ctx.database().checkpointReadUnlock();
         }
     }
 
@@ -2188,27 +2212,34 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public void onEvicted(GridDhtLocalPartition part, boolean updateSeq) {
-        lock.writeLock().lock();
+        ctx.database().checkpointReadLock();
 
         try {
-            if (stopping)
-                return;
+            lock.writeLock().lock();
 
-            assert part.state() == EVICTED;
+            try {
+                if (stopping)
+                    return;
 
-            long seq = updateSeq ? this.updateSeq.incrementAndGet() : this.updateSeq.get();
+                assert part.state() == EVICTED;
 
-            if (part.reload())
-                part = createPartition(part.id());
+                long seq = updateSeq ? this.updateSeq.incrementAndGet() : this.updateSeq.get();
 
-            assert lastTopChangeVer.initialized() : lastTopChangeVer;
+                if (part.reload())
+                    part = createPartition(part.id());
 
-            updateLocal(part.id(), part.state(), seq, lastTopChangeVer);
+                assert lastTopChangeVer.initialized() : lastTopChangeVer;
 
-            consistencyCheck();
+                updateLocal(part.id(), part.state(), seq, lastTopChangeVer);
+
+                consistencyCheck();
+            }
+            finally {
+                lock.writeLock().unlock();
+            }
         }
         finally {
-            lock.writeLock().unlock();
+            ctx.database().checkpointReadUnlock();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21de1c56/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 1b5dae6..85e3baa 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
@@ -2281,6 +2281,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 for (DbCheckpointListener lsnr : lsnrs)
                     lsnr.onCheckpointBegin(ctx0);
 
+                if (curr.nextSnapshot)
+                    snapshotMgr.onMarkCheckPointBegin(curr.snapshotOperation, map);
+
                 for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
                     if (grp.isLocal())
                         continue;
@@ -2300,8 +2303,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     cpRec.addCacheGroupState(grp.groupId(), state);
                 }
 
-                if (curr.nextSnapshot)
-                    snapshotMgr.onMarkCheckPointBegin(curr.snapshotOperation, map);
 
                 cpPagesTuple = beginAllCheckpoints();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21de1c56/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
new file mode 100644
index 0000000..3969fb6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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;
+
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.U;
+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;
+
+/**
+ *
+ */
+public class IgnitePdsExchangeDuringCheckpointTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /**
+     *
+     */
+    public void testExchangeOnNodeLeft() throws Exception {
+        for (int i = 0; i < 5; i++) {
+            startGrids(3);
+            IgniteEx ignite = grid(1);
+            ignite.active(true);
+
+            awaitPartitionMapExchange();
+
+            stopGrid(0, true);
+
+            awaitPartitionMapExchange();
+
+            ignite.context().cache().context().database().wakeupForCheckpoint("test").get(10000);
+
+            afterTest();
+        }
+    }
+
+    /**
+     *
+     */
+    public void testExchangeOnNodeJoin() throws Exception {
+        for (int i = 0; i < 5; i++) {
+            startGrids(2);
+            IgniteEx ignite = grid(1);
+            ignite.active(true);
+
+            awaitPartitionMapExchange();
+
+            IgniteEx ex = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            ex.context().cache().context().database().wakeupForCheckpoint("test").get(10000);
+
+            afterTest();
+        }
+    }
+
+    /**
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        MemoryConfiguration memCfg = new MemoryConfiguration();
+
+        MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
+
+        memPlcCfg.setName("dfltMemPlc");
+        memPlcCfg.setInitialSize(100 * 1024 * 1024);
+        memPlcCfg.setMaxSize(1000 * 1024 * 1024);
+
+        memCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+        memCfg.setMemoryPolicies(memPlcCfg);
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 4096));
+
+        cfg.setCacheConfiguration(ccfg);
+
+        PersistentStoreConfiguration psiCfg = new PersistentStoreConfiguration()
+            .setCheckpointingThreads(1)
+            .setCheckpointingFrequency(1)
+            .setWalMode(WALMode.LOG_ONLY);
+
+        cfg.setPersistentStoreConfiguration(psiCfg);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/21de1c56/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index 568af94..6e16d2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDa
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDiscoveryDataStructuresTest;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteDataStructureUniqueNameTest;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteDataStructureWithJobTest;
+import org.apache.ignite.internal.processors.cache.datastructures.SemaphoreFailoverSafeReleasePermitsTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalAtomicQueueApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalAtomicSetSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalQueueApiSelfTest;
@@ -129,8 +130,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgnitePartitionedCountDownLatchSelfTest.class));
         suite.addTest(new TestSuite(IgniteDataStructureWithJobTest.class));
         suite.addTest(new TestSuite(IgnitePartitionedSemaphoreSelfTest.class));
-        // TODO https://issues.apache.org/jira/browse/IGNITE-4173, enable when fixed.
-        // suite.addTest(new TestSuite(SemaphoreFailoverSafeReleasePermitsTest.class));
+        suite.addTest(new TestSuite(SemaphoreFailoverSafeReleasePermitsTest.class));
         // TODO IGNITE-3141, enabled when fixed.
         // suite.addTest(new TestSuite(IgnitePartitionedLockSelfTest.class));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21de1c56/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 ab8ff81..29ea64c 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
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes;
+import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsExchangeDuringCheckpointTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistenceMetricsSelfTest;
@@ -73,6 +74,9 @@ public class IgnitePdsTestSuite2 extends TestSuite {
         suite.addTestSuite(IgniteWalFlushFailoverTest.class);
 
         suite.addTestSuite(IgniteWalReaderTest.class);
+
+        suite.addTestSuite(IgnitePdsExchangeDuringCheckpointTest.class);
+
         return suite;
     }
 }


[07/50] [abbrv] ignite git commit: IGNITE-6399 .NET: Added ClientConnectorConfiguration. This closes #2696.

Posted by yz...@apache.org.
IGNITE-6399 .NET: Added ClientConnectorConfiguration. This closes #2696.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 915682bd5facee9dd62fc5d6dfb35e37ff601db8
Parents: 718e365
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Sep 19 16:32:07 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Sep 19 16:33:06 2017 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  53 +++++++
 .../Client/Cache/ScanQueryTest.cs               |   2 +-
 .../Client/ClientConnectionTest.cs              |  26 ++-
 .../Client/RawSocketTest.cs                     |   4 +-
 .../IgniteConfigurationSerializerTest.cs        |  17 +-
 .../IgniteConfigurationTest.cs                  |  40 +++++
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Client/IgniteClientConfiguration.cs         |   2 +-
 .../ClientConnectorConfiguration.cs             | 159 +++++++++++++++++++
 .../Configuration/SqlConnectorConfiguration.cs  |   2 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  50 +++++-
 .../IgniteConfigurationSection.xsd              |  52 ++++++
 12 files changed, 400 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/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 d54aac1..7e17bdb 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
@@ -53,6 +53,7 @@ import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
 import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
@@ -682,6 +683,12 @@ public class PlatformConfigurationUtils {
             cfg.setSqlConnectorConfiguration(readSqlConnectorConfiguration(in));
 
         if (in.readBoolean())
+            cfg.setClientConnectorConfiguration(readClientConnectorConfiguration(in));
+
+        if (!in.readBoolean())  // ClientConnectorConfigurationEnabled override
+            cfg.setClientConnectorConfiguration(null);
+
+        if (in.readBoolean())
             cfg.setPersistentStoreConfiguration(readPersistentStoreConfiguration(in));
 
         readPluginConfiguration(cfg, in);
@@ -1147,6 +1154,10 @@ public class PlatformConfigurationUtils {
 
         writeSqlConnectorConfiguration(w, cfg.getSqlConnectorConfiguration());
 
+        writeClientConnectorConfiguration(w, cfg.getClientConnectorConfiguration());
+
+        w.writeBoolean(cfg.getClientConnectorConfiguration() != null);
+
         writePersistentStoreConfiguration(w, cfg.getPersistentStoreConfiguration());
 
         w.writeString(cfg.getIgniteHome());
@@ -1483,6 +1494,48 @@ public class PlatformConfigurationUtils {
     }
 
     /**
+     * Reads the client connector configuration.
+     *
+     * @param in Reader.
+     * @return Config.
+     */
+    private static ClientConnectorConfiguration readClientConnectorConfiguration(BinaryRawReader in) {
+        return new ClientConnectorConfiguration()
+                .setHost(in.readString())
+                .setPort(in.readInt())
+                .setPortRange(in.readInt())
+                .setSocketSendBufferSize(in.readInt())
+                .setSocketReceiveBufferSize(in.readInt())
+                .setTcpNoDelay(in.readBoolean())
+                .setMaxOpenCursorsPerConnection(in.readInt())
+                .setThreadPoolSize(in.readInt());
+    }
+
+    /**
+     * Writes the client connector configuration.
+     *
+     * @param w Writer.
+     */
+    private static void writeClientConnectorConfiguration(BinaryRawWriter w, ClientConnectorConfiguration cfg) {
+        assert w != null;
+
+        if (cfg != null) {
+            w.writeBoolean(true);
+
+            w.writeString(cfg.getHost());
+            w.writeInt(cfg.getPort());
+            w.writeInt(cfg.getPortRange());
+            w.writeInt(cfg.getSocketSendBufferSize());
+            w.writeInt(cfg.getSocketReceiveBufferSize());
+            w.writeBoolean(cfg.isTcpNoDelay());
+            w.writeInt(cfg.getMaxOpenCursorsPerConnection());
+            w.writeInt(cfg.getThreadPoolSize());
+        } else {
+            w.writeBoolean(false);
+        }
+    }
+
+    /**
      * Reads the persistence store connector configuration.
      *
      * @param in Reader.

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/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 bc1e8ee..7f8b589 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
@@ -46,7 +46,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         {
             var cfg = base.GetIgniteConfiguration();
 
-            cfg.SqlConnectorConfiguration = new SqlConnectorConfiguration
+            cfg.ClientConnectorConfiguration = new ClientConnectorConfiguration
             {
                 MaxOpenCursorsPerConnection = 3
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 8874bb5..7dcec55 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -79,7 +79,7 @@ namespace Apache.Ignite.Core.Tests.Client
         {
             var servCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
             {
-                SqlConnectorConfiguration = new SqlConnectorConfiguration
+                ClientConnectorConfiguration = new ClientConnectorConfiguration
                 {
                     Host = "localhost",
                     Port = 2000,
@@ -128,6 +128,30 @@ namespace Apache.Ignite.Core.Tests.Client
         }
 
         /// <summary>
+        /// Tests that connector can be disabled.
+        /// </summary>
+        [Test]
+        public void TestDisabledConnector()
+        {
+            var servCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                ClientConnectorConfigurationEnabled = false
+            };
+
+            var clientCfg = new IgniteClientConfiguration
+            {
+                Host = "localhost"
+            };
+
+            using (Ignition.Start(servCfg))
+            {
+                var ex = Assert.Throws<AggregateException>(() => Ignition.StartClient(clientCfg));
+                Assert.AreEqual("Failed to establish Ignite thin client connection, " +
+                                "examine inner exceptions for details.", ex.Message);
+            }
+        }
+
+        /// <summary>
         /// Starts the client.
         /// </summary>
         private static IIgniteClient StartClient()

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
index 48bd3dd..b34e037 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
@@ -40,7 +40,7 @@ namespace Apache.Ignite.Core.Tests.Client
         {
             var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
             {
-                SqlConnectorConfiguration = new SqlConnectorConfiguration()
+                ClientConnectorConfiguration = new ClientConnectorConfiguration()
             };
 
             using (var ignite = Ignition.Start(cfg))
@@ -53,7 +53,7 @@ namespace Apache.Ignite.Core.Tests.Client
                 cache[1] = "bar";
 
                 // Connect socket.
-                var sock = GetSocket(SqlConnectorConfiguration.DefaultPort);
+                var sock = GetSocket(ClientConnectorConfiguration.DefaultPort);
                 Assert.IsTrue(sock.Connected);
 
                 DoHandshake(sock);

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 125902f..ac214ce 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -143,6 +143,7 @@ namespace Apache.Ignite.Core.Tests
                                 </memoryPolicies>
                             </memoryConfiguration>
                             <sqlConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
+                            <clientConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
                             <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' walStorePath='baz' metricsEnabled='true' rateTimeInterval='0:0:6' subIntervals='3' />
                         </igniteConfig>";
 
@@ -285,6 +286,7 @@ namespace Apache.Ignite.Core.Tests
 
             Assert.AreEqual(PeerAssemblyLoadingMode.CurrentAppDomain, cfg.PeerAssemblyLoadingMode);
 
+#pragma warning disable 618  // Obsolete
             var sql = cfg.SqlConnectorConfiguration;
             Assert.IsNotNull(sql);
             Assert.AreEqual("bar", sql.Host);
@@ -295,6 +297,18 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsTrue(sql.TcpNoDelay);
             Assert.AreEqual(14, sql.MaxOpenCursorsPerConnection);
             Assert.AreEqual(15, sql.ThreadPoolSize);
+#pragma warning restore 618
+
+            var client = cfg.ClientConnectorConfiguration;
+            Assert.IsNotNull(client);
+            Assert.AreEqual("bar", client.Host);
+            Assert.AreEqual(10, client.Port);
+            Assert.AreEqual(11, client.PortRange);
+            Assert.AreEqual(12, client.SocketSendBufferSize);
+            Assert.AreEqual(13, client.SocketReceiveBufferSize);
+            Assert.IsTrue(client.TcpNoDelay);
+            Assert.AreEqual(14, client.MaxOpenCursorsPerConnection);
+            Assert.AreEqual(15, client.ThreadPoolSize);
 
             var pers = cfg.PersistentStoreConfiguration;
 
@@ -355,6 +369,7 @@ namespace Apache.Ignite.Core.Tests
         /// Checks the property is present in schema.
         /// </summary>
         // ReSharper disable once UnusedParameter.Local
+        // ReSharper disable once ParameterOnlyUsedForPreconditionCheck.Local
         private static void CheckPropertyIsPresentInSchema(Type type, XElement schema)
         {
             Func<string, string> toLowerCamel = x => char.ToLowerInvariant(x[0]) + x.Substring(1);
@@ -879,7 +894,7 @@ namespace Apache.Ignite.Core.Tests
                     }
                 },
                 PeerAssemblyLoadingMode = PeerAssemblyLoadingMode.CurrentAppDomain,
-                SqlConnectorConfiguration = new SqlConnectorConfiguration
+                ClientConnectorConfiguration = new ClientConnectorConfiguration
                 {
                     Host = "foo",
                     Port = 2,

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/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 950f36d..5facb38 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -63,6 +63,8 @@ namespace Apache.Ignite.Core.Tests
         {
             CheckDefaultProperties(new IgniteConfiguration());
             CheckDefaultProperties(new PersistentStoreConfiguration());
+            CheckDefaultProperties(new ClientConnectorConfiguration());
+            CheckDefaultProperties(new SqlConnectorConfiguration());
         }
 
         /// <summary>
@@ -87,6 +89,7 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new MemoryConfiguration());
             CheckDefaultValueAttributes(new MemoryPolicyConfiguration());
             CheckDefaultValueAttributes(new SqlConnectorConfiguration());
+            CheckDefaultValueAttributes(new ClientConnectorConfiguration());
             CheckDefaultValueAttributes(new PersistentStoreConfiguration());
             CheckDefaultValueAttributes(new IgniteClientConfiguration());
         }
@@ -321,6 +324,9 @@ namespace Apache.Ignite.Core.Tests
 
                 // Check PersistentStoreConfiguration defaults.
                 CheckDefaultProperties(resCfg.PersistentStoreConfiguration);
+
+                // Connector defaults.
+                CheckDefaultProperties(resCfg.ClientConnectorConfiguration);
             }
         }
 
@@ -515,6 +521,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(IgniteConfiguration.DefaultLongQueryWarningTimeout, cfg.LongQueryWarningTimeout);
             Assert.AreEqual(IgniteConfiguration.DefaultIsLateAffinityAssignment, cfg.IsLateAffinityAssignment);
             Assert.AreEqual(IgniteConfiguration.DefaultIsActiveOnStart, cfg.IsActiveOnStart);
+            Assert.AreEqual(IgniteConfiguration.DefaultClientConnectorConfigurationEnabled, 
+                cfg.ClientConnectorConfigurationEnabled);
 
             // Thread pools.
             Assert.AreEqual(IgniteConfiguration.DefaultManagementThreadPoolSize, cfg.ManagementThreadPoolSize);
@@ -555,6 +563,38 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
+        /// Checks the default properties.
+        /// </summary>
+        /// <param name="cfg">Config.</param>
+        private static void CheckDefaultProperties(ClientConnectorConfiguration cfg)
+        {
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultPort, cfg.Port);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultPortRange, cfg.PortRange);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultMaxOpenCursorsPerConnection,
+                cfg.MaxOpenCursorsPerConnection);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultSocketBufferSize, cfg.SocketReceiveBufferSize);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultSocketBufferSize, cfg.SocketSendBufferSize);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultTcpNoDelay, cfg.TcpNoDelay);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultThreadPoolSize, cfg.ThreadPoolSize);
+        }
+
+        /// <summary>
+        /// Checks the default properties.
+        /// </summary>
+        /// <param name="cfg">Config.</param>
+        private static void CheckDefaultProperties(SqlConnectorConfiguration cfg)
+        {
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultPort, cfg.Port);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultPortRange, cfg.PortRange);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultMaxOpenCursorsPerConnection,
+                cfg.MaxOpenCursorsPerConnection);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultSocketBufferSize, cfg.SocketReceiveBufferSize);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultSocketBufferSize, cfg.SocketSendBufferSize);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultTcpNoDelay, cfg.TcpNoDelay);
+            Assert.AreEqual(ClientConnectorConfiguration.DefaultThreadPoolSize, cfg.ThreadPoolSize);
+        }
+
+        /// <summary>
         /// Checks the default value attributes.
         /// </summary>
         /// <param name="obj">The object.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index c74c538..75da6c8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -102,6 +102,7 @@
     <Compile Include="Client\IIgniteClient.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
+    <Compile Include="Configuration\ClientConnectorConfiguration.cs" />
     <Compile Include="Impl\Binary\BinaryTypeId.cs" />
     <Compile Include="Impl\Client\Cache\CacheFlags.cs" />
     <Compile Include="Impl\Client\Cache\Query\ClientQueryCursor.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
index 3339c65..8c9b6a1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
@@ -25,7 +25,7 @@ namespace Apache.Ignite.Core.Client
     /// Ignite thin client configuration.
     /// <para />
     /// Ignite thin client connects to a specific Ignite node with a socket and does not start JVM in process.
-    /// This configuration should correspond to <see cref="IgniteConfiguration.SqlConnectorConfiguration"/>
+    /// This configuration should correspond to <see cref="IgniteConfiguration.ClientConnectorConfiguration"/>
     /// on a target node.
     /// </summary>
     public class IgniteClientConfiguration

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
new file mode 100644
index 0000000..8c23d99
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Configuration
+{
+    using System.ComponentModel;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// Client connector configuration (ODBC, JDBC, Thin Client).
+    /// </summary>
+    public class ClientConnectorConfiguration
+    {
+        /// <summary>
+        /// Default port.
+        /// </summary>
+        public const int DefaultPort = 10800;
+
+        /// <summary>
+        /// Default port range.
+        /// </summary>
+        public const int DefaultPortRange = 100;
+
+        /// <summary>
+        /// Default socket buffer size.
+        /// </summary>
+        public const int DefaultSocketBufferSize = 0;
+
+        /// <summary>
+        /// Default value of <see cref="TcpNoDelay" /> property.
+        /// </summary>
+        public const bool DefaultTcpNoDelay = true;
+
+        /// <summary>
+        /// Default maximum number of open cursors per connection.
+        /// </summary>
+        public const int DefaultMaxOpenCursorsPerConnection = 128;
+
+        /// <summary>
+        /// Default SQL connector thread pool size.
+        /// </summary>
+        public static readonly int DefaultThreadPoolSize = IgniteConfiguration.DefaultThreadPoolSize;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientConnectorConfiguration"/> class.
+        /// </summary>
+        public ClientConnectorConfiguration()
+        {
+            Port = DefaultPort;
+            PortRange = DefaultPortRange;
+            SocketSendBufferSize = DefaultSocketBufferSize;
+            SocketReceiveBufferSize = DefaultSocketBufferSize;
+            TcpNoDelay = DefaultTcpNoDelay;
+            MaxOpenCursorsPerConnection = DefaultMaxOpenCursorsPerConnection;
+            ThreadPoolSize = DefaultThreadPoolSize;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientConnectorConfiguration"/> class.
+        /// </summary>
+        internal ClientConnectorConfiguration(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            Host = reader.ReadString();
+            Port = reader.ReadInt();
+            PortRange = reader.ReadInt();
+            SocketSendBufferSize = reader.ReadInt();
+            SocketReceiveBufferSize = reader.ReadInt();
+            TcpNoDelay = reader.ReadBoolean();
+            MaxOpenCursorsPerConnection = reader.ReadInt();
+            ThreadPoolSize = reader.ReadInt();
+        }
+
+        /// <summary>
+        /// Writes to the specified writer.
+        /// </summary>
+        internal void Write(IBinaryRawWriter writer)
+        {
+            Debug.Assert(writer != null);
+            
+            writer.WriteString(Host);
+            writer.WriteInt(Port);
+            writer.WriteInt(PortRange);
+            writer.WriteInt(SocketSendBufferSize);
+            writer.WriteInt(SocketReceiveBufferSize);
+            writer.WriteBoolean(TcpNoDelay);
+            writer.WriteInt(MaxOpenCursorsPerConnection);
+            writer.WriteInt(ThreadPoolSize);
+        }
+
+        /// <summary>
+        /// Gets or sets the host.
+        /// </summary>
+        public string Host { get; set; }
+
+        /// <summary>
+        /// Gets or sets the port.
+        /// </summary>
+        [DefaultValue(DefaultPort)]
+        public int Port { get; set; }
+
+        /// <summary>
+        /// Gets or sets the port range.
+        /// </summary>
+        [DefaultValue(DefaultPortRange)]
+        public int PortRange { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the socket send buffer. When set to 0, operating system default is used.
+        /// </summary>
+        [DefaultValue(DefaultSocketBufferSize)]
+        public int SocketSendBufferSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the socket receive buffer. When set to 0, operating system default is used.
+        /// </summary>
+        [DefaultValue(DefaultSocketBufferSize)]
+        public int SocketReceiveBufferSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the value for <c>TCP_NODELAY</c> socket option. Each
+        /// socket will be opened using provided value.
+        /// <para />
+        /// Setting this option to <c>true</c> disables Nagle's algorithm
+        /// for socket decreasing latency and delivery time for small messages.
+        /// <para />
+        /// For systems that work under heavy network load it is advisable to set this value to <c>false</c>.
+        /// </summary>
+        [DefaultValue(DefaultTcpNoDelay)]
+        public bool TcpNoDelay { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum open cursors per connection.
+        /// </summary>
+        [DefaultValue(DefaultMaxOpenCursorsPerConnection)]
+        public int MaxOpenCursorsPerConnection { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the thread pool.
+        /// </summary>
+        public int ThreadPoolSize { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/SqlConnectorConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/SqlConnectorConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/SqlConnectorConfiguration.cs
index 0a9b7e5..dd262ec 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/SqlConnectorConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/SqlConnectorConfiguration.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Configuration
 {
+    using System;
     using System.ComponentModel;
     using System.Diagnostics;
     using Apache.Ignite.Core.Binary;
@@ -24,6 +25,7 @@ namespace Apache.Ignite.Core.Configuration
     /// <summary>
     /// SQL connector configuration (for ODBC and JDBC).
     /// </summary>
+    [Obsolete("Use ClientConnectorConfiguration instead.")]
     public class SqlConnectorConfiguration
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/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 4d04348..5ac2258 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -121,6 +121,11 @@ namespace Apache.Ignite.Core
         /// </summary>
         public static readonly TimeSpan DefaultLongQueryWarningTimeout = TimeSpan.FromMilliseconds(3000);
 
+        /// <summary>
+        /// Default value for <see cref="ClientConnectorConfigurationEnabled"/>.
+        /// </summary>
+        public const bool DefaultClientConnectorConfigurationEnabled = true;
+
         /** */
         private TimeSpan? _metricsExpireTime;
 
@@ -209,6 +214,7 @@ namespace Apache.Ignite.Core
         {
             JvmInitialMemoryMb = DefaultJvmInitMem;
             JvmMaxMemoryMb = DefaultJvmMaxMem;
+            ClientConnectorConfigurationEnabled = DefaultClientConnectorConfigurationEnabled;
         }
 
         /// <summary>
@@ -436,17 +442,32 @@ namespace Apache.Ignite.Core
                 writer.WriteBoolean(false);
             }
 
-            // SQL
+            // SQL connector.
+#pragma warning disable 618  // Obsolete
             if (SqlConnectorConfiguration != null)
             {
                 writer.WriteBoolean(true);
                 SqlConnectorConfiguration.Write(writer);
             }
+#pragma warning restore 618
+            else
+            {
+                writer.WriteBoolean(false);
+            }
+
+            // Client connector.
+            if (ClientConnectorConfiguration != null)
+            {
+                writer.WriteBoolean(true);
+                ClientConnectorConfiguration.Write(writer);
+            }
             else
             {
                 writer.WriteBoolean(false);
             }
 
+            writer.WriteBoolean(ClientConnectorConfigurationEnabled);
+
             // Persistence.
             if (PersistentStoreConfiguration != null)
             {
@@ -609,12 +630,22 @@ namespace Apache.Ignite.Core
                 MemoryConfiguration = new MemoryConfiguration(r);
             }
 
-            // SQL
+            // SQL.
             if (r.ReadBoolean())
             {
+#pragma warning disable 618  // Obsolete
                 SqlConnectorConfiguration = new SqlConnectorConfiguration(r);
+#pragma warning restore 618
             }
 
+            // Client.
+            if (r.ReadBoolean())
+            {
+                ClientConnectorConfiguration = new ClientConnectorConfiguration(r);
+            }
+
+            ClientConnectorConfigurationEnabled = r.ReadBoolean();
+
             // Persistence.
             if (r.ReadBoolean())
             {
@@ -1196,9 +1227,24 @@ namespace Apache.Ignite.Core
         /// <summary>
         /// Gets or sets the SQL connector configuration (for JDBC and ODBC).
         /// </summary>
+        [Obsolete("Use ClientConnectorConfiguration instead.")]
         public SqlConnectorConfiguration SqlConnectorConfiguration { get; set; }
 
         /// <summary>
+        /// Gets or sets the client connector configuration (for JDBC, ODBC, and thin clients).
+        /// </summary>
+        public ClientConnectorConfiguration ClientConnectorConfiguration { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether client connector is enabled:
+        /// allow thin clients, ODBC and JDBC drivers to work with Ignite
+        /// (see <see cref="ClientConnectorConfiguration"/>).
+        /// Default is <see cref="DefaultClientConnectorConfigurationEnabled"/>.
+        /// </summary>
+        [DefaultValue(DefaultClientConnectorConfigurationEnabled)]
+        public bool ClientConnectorConfigurationEnabled { get; set; }
+
+        /// <summary>
         /// Gets or sets the timeout after which long query warning will be printed.
         /// </summary>
         [DefaultValue(typeof(TimeSpan), "00:00:03")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/915682bd/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 d7fd5ac..ac1111b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1309,6 +1309,53 @@
                         </xs:attribute>
                     </xs:complexType>
                 </xs:element>
+                <xs:element name="clientConnectorConfiguration" minOccurs="0">
+                    <xs:annotation>
+                        <xs:documentation>Client connector configuration (JDBC, ODBC, thin clients).</xs:documentation>
+                    </xs:annotation>
+                    <xs:complexType>
+                        <xs:attribute name="port" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Connector port.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="host" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Connector host.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="portRange" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Connector port range.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="socketSendBufferSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the socket send buffer.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="socketReceiveBufferSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the socket receive buffer.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="tcpNoDelay" type="xs:boolean">
+                            <xs:annotation>
+                                <xs:documentation>TCP_NODELAY socket option.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="maxOpenCursorsPerConnection" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Maximum open cursors per connection.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="threadPoolSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>SQL connector thread pool size.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                    </xs:complexType>
+                </xs:element>
                 <xs:element name="persistentStoreConfiguration" minOccurs="0">
                     <xs:annotation>
                         <xs:documentation>Persistent store configuration.</xs:documentation>
@@ -1653,6 +1700,11 @@
                     <xs:documentation>Whether grid should be active on start.</xs:documentation>
                 </xs:annotation>
             </xs:attribute>
+            <xs:attribute name="clientConnectorConfigurationEnabled" type="xs:boolean">
+                <xs:annotation>
+                    <xs:documentation>Whether client connector should be enabled (allow thin clients, ODBC and JDBC drivers to work with Ignite).</xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
         </xs:complexType>
     </xs:element>
 </xs:schema>


[24/50] [abbrv] ignite git commit: Merge remote-tracking branch 'professional/ignite-2.1.5' into ignite-2.1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'professional/ignite-2.1.5' into ignite-2.1.5


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: e82f077df78972a58fdd06917839445711432705
Parents: aeb4aea 21d7fed
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Thu Sep 21 18:56:40 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Thu Sep 21 18:56:40 2017 +0300

----------------------------------------------------------------------
 .../pagemem/store/IgnitePageStoreManager.java   |   4 +-
 .../cache/CacheAffinitySharedManager.java       |   4 +-
 .../persistence/file/FilePageStoreManager.java  |   7 +-
 .../processors/query/GridQueryProcessor.java    |  30 ++-
 .../pagemem/NoOpPageStoreManager.java           |   2 +-
 .../IgnitePersistentStoreSchemaLoadTest.java    | 237 ++++++++++++-------
 6 files changed, 184 insertions(+), 100 deletions(-)
----------------------------------------------------------------------



[03/50] [abbrv] ignite git commit: IGNITE-6428 Fixed IgniteOOME in PDS Indexing suite (increased memory policy size) - Fixes #2690.

Posted by yz...@apache.org.
IGNITE-6428 Fixed IgniteOOME in PDS Indexing suite (increased memory policy size) - Fixes #2690.

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/be60c086
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/be60c086
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/be60c086

Branch: refs/heads/ignite-2.1.5-p1
Commit: be60c08687f873a278d882666b61d60af7c0205c
Parents: 4a1e90c
Author: dpavlov <dp...@gridgain.com>
Authored: Tue Sep 19 15:23:18 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Sep 19 15:30:42 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be60c086/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
index 2d237cb..96f832b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
@@ -120,7 +120,7 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb
         MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
 
         memPlcCfg.setName("dfltMemPlc");
-        memPlcCfg.setMaxSize(100 * 1024 * 1024);
+        memPlcCfg.setMaxSize(150 * 1024 * 1024);
         memPlcCfg.setInitialSize(100 * 1024 * 1024);
         memPlcCfg.setSwapFilePath("work/swap");
 


[09/50] [abbrv] ignite git commit: IGNITE-6376 Web console: Enable task and job events in demo mode by default. (cherry picked from commit ff17783)

Posted by yz...@apache.org.
IGNITE-6376 Web console: Enable task and job events in demo mode by default.
(cherry picked from commit ff17783)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 6318c1a15f05e952c22e42e6e238816d7ffdb0ca
Parents: 4338799
Author: vsisko <vs...@gridgain.com>
Authored: Wed Sep 20 17:19:58 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 20 17:25:00 2017 +0700

----------------------------------------------------------------------
 .../internal/visor/node/VisorNodeDataCollectorTask.java     | 2 +-
 .../visor/node/VisorNodeDataCollectorTaskResult.java        | 4 ++--
 modules/web-console/backend/routes/demo/domains.json        | 5 +++++
 .../configuration/generator/JavaTransformer.service.js      | 5 +++--
 .../org/apache/ignite/console/demo/AgentClusterDemo.java    | 9 ++++++++-
 5 files changed, 19 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6318c1a1/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index 80664a1..abd9ce2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -89,7 +89,7 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
 
         taskRes.getTopologyVersions().put(nid, jobRes.getTopologyVersion());
 
-        taskRes.isTaskMonitoringEnabled().put(nid, jobRes.isTaskMonitoringEnabled());
+        taskRes.getTaskMonitoringEnabled().put(nid, jobRes.isTaskMonitoringEnabled());
 
         taskRes.getErrorCounts().put(nid, jobRes.getErrorCount());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6318c1a1/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
index c1e2f1f..6e10b84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
@@ -169,7 +169,7 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
     /**
      * @return All task monitoring state collected from nodes.
      */
-    public Map<UUID, Boolean> isTaskMonitoringEnabled() {
+    public Map<UUID, Boolean> getTaskMonitoringEnabled() {
         return taskMonitoringEnabled;
     }
 
@@ -283,7 +283,7 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         unhandledEx.putAll(res.getUnhandledEx());
         gridNames.putAll(res.getGridNames());
         topVersions.putAll(res.getTopologyVersions());
-        taskMonitoringEnabled.putAll(res.isTaskMonitoringEnabled());
+        taskMonitoringEnabled.putAll(res.getTaskMonitoringEnabled());
         errCnts.putAll(res.getErrorCounts());
         evts.addAll(res.getEvents());
         evtsEx.putAll(res.getEventsEx());

http://git-wip-us.apache.org/repos/asf/ignite/blob/6318c1a1/modules/web-console/backend/routes/demo/domains.json
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/demo/domains.json b/modules/web-console/backend/routes/demo/domains.json
index ae46758..25f5019 100644
--- a/modules/web-console/backend/routes/demo/domains.json
+++ b/modules/web-console/backend/routes/demo/domains.json
@@ -17,6 +17,7 @@
         "className": "Integer"
       }
     ],
+    "keyFieldName": "id",
     "valueFields": [
       {
         "databaseFieldName": "NAME",
@@ -60,6 +61,7 @@
         "className": "String"
       }
     ],
+    "keyFieldName": "id",
     "valueFields": [
       {
         "databaseFieldName": "COUNTRY_ID",
@@ -156,6 +158,7 @@
         "className": "Double"
       }
     ],
+    "keyFieldName": "id",
     "valueFields": [
       {
         "databaseFieldName": "DEPARTMENT_ID",
@@ -241,6 +244,7 @@
         "className": "Integer"
       }
     ],
+    "keyFieldName": "id",
     "valueFields": [
       {
         "databaseFieldName": "NAME",
@@ -284,6 +288,7 @@
         "className": "String"
       }
     ],
+    "keyFieldName": "id",
     "valueFields": [
       {
         "databaseFieldName": "PARKING_ID",

http://git-wip-us.apache.org/repos/asf/ignite/blob/6318c1a1/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
index 64fc172..4e6204f 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
@@ -1606,6 +1606,9 @@ export default class IgniteJavaTransformer extends AbstractTransformer {
             shortFactoryCls = this.javaTypes.shortClassName(factoryCls);
         }
 
+        if ((_.nonEmpty(clientNearCaches) || demo) && shortFactoryCls)
+            imports.push('org.apache.ignite.Ignite');
+
         sb.append(`package ${pkg};`)
             .emptyLine();
 
@@ -1651,8 +1654,6 @@ export default class IgniteJavaTransformer extends AbstractTransformer {
         }
 
         if ((_.nonEmpty(clientNearCaches) || demo) && shortFactoryCls) {
-            imports.push('org.apache.ignite.Ignite');
-
             sb.append(`Ignite ignite = Ignition.start(${cfgRef});`);
 
             _.forEach(clientNearCaches, (cache, idx) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6318c1a1/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
index 73577b5..886888b 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
@@ -55,6 +55,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
 import static org.apache.ignite.console.demo.AgentDemoUtils.newScheduledThreadPool;
 import static org.apache.ignite.events.EventType.EVTS_DISCOVERY;
+import static org.apache.ignite.internal.visor.util.VisorTaskUtils.VISOR_TASK_EVTS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_JETTY_ADDRS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_JETTY_PORT;
 
@@ -92,7 +93,13 @@ public class AgentClusterDemo {
         cfg.setIgniteInstanceName((client ? "demo-client-" : "demo-server-" ) + gridIdx);
         cfg.setLocalHost("127.0.0.1");
         cfg.setEventStorageSpi(new MemoryEventStorageSpi());
-        cfg.setIncludeEventTypes(EVTS_DISCOVERY);
+
+        int[] evts = new int[EVTS_DISCOVERY.length + VISOR_TASK_EVTS.length];
+
+        System.arraycopy(EVTS_DISCOVERY, 0, evts, 0, EVTS_DISCOVERY.length);
+        System.arraycopy(VISOR_TASK_EVTS, 0, evts, EVTS_DISCOVERY.length, VISOR_TASK_EVTS.length);
+
+        cfg.setIncludeEventTypes(evts);
 
         cfg.getConnectorConfiguration().setPort(basePort);
 


[20/50] [abbrv] ignite git commit: IGNITE-6465: JDBC thin driver: added SQLSTATE propagation for BatchUpdateException. This closes #2719.

Posted by yz...@apache.org.
IGNITE-6465: JDBC thin driver: added SQLSTATE propagation for BatchUpdateException. This closes #2719.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: be8afd4f39599b7e26ad148331c5aa6ba4faf513
Parents: 099d33c
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Sep 21 17:51:26 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Thu Sep 21 17:52:17 2017 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinErrorsSelfTest.java       | 31 ++++++++++++++++++++
 .../internal/jdbc/thin/JdbcThinStatement.java   |  7 +++--
 2 files changed, 36 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be8afd4f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinErrorsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinErrorsSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinErrorsSelfTest.java
index afd06ed..db70f3be 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinErrorsSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinErrorsSelfTest.java
@@ -17,9 +17,11 @@
 
 package org.apache.ignite.jdbc.thin;
 
+import java.sql.BatchUpdateException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
+import java.sql.Statement;
 import org.apache.ignite.jdbc.JdbcErrorsAbstractSelfTest;
 import org.apache.ignite.lang.IgniteCallable;
 
@@ -74,4 +76,33 @@ public class JdbcThinErrorsSelfTest extends JdbcErrorsAbstractSelfTest {
             }
         }, "0700E");
     }
+
+    /**
+     * Test error code for the case when error is caused on batch execution.
+     * @throws SQLException if failed.
+     */
+    @SuppressWarnings("MagicConstant")
+    public void testBatchUpdateException() throws SQLException {
+        try (final Connection conn = getConnection()) {
+            try (Statement stmt = conn.createStatement()) {
+                stmt.executeUpdate("CREATE TABLE test (id int primary key, val varchar)");
+
+                stmt.addBatch("insert into test (id, val) values (1, 'val1')");
+                stmt.addBatch("insert into test (id, val) values (2, 'val2')");
+                stmt.addBatch("insert into test (id1, val1) values (3, 'val3')");
+
+                stmt.executeBatch();
+
+                fail("BatchUpdateException is expected");
+            }
+            catch (BatchUpdateException e) {
+                assertEquals(2, e.getUpdateCounts().length);
+
+                for (int updCnt : e.getUpdateCounts())
+                    assertEquals(1, updCnt);
+
+                assertEquals("42000", e.getSQLState());
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/be8afd4f/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
index 6ab50de..8e096c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
@@ -27,6 +27,7 @@ import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest;
@@ -371,8 +372,10 @@ public class JdbcThinStatement implements Statement {
         try {
             JdbcBatchExecuteResult res = conn.sendRequest(new JdbcBatchExecuteRequest(conn.getSchema(), batch));
 
-            if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS)
-                throw new BatchUpdateException(res.errorMessage(), null, res.errorCode(), res.updateCounts());
+            if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS) {
+                throw new BatchUpdateException(res.errorMessage(), IgniteQueryErrorCode.codeToSqlState(res.errorCode()),
+                    res.errorCode(), res.updateCounts());
+            }
 
             return res.updateCounts();
         }


[43/50] [abbrv] ignite git commit: IGNITE-6219 - IgniteCache#loadCache executes local load in caller thread

Posted by yz...@apache.org.
IGNITE-6219 - IgniteCache#loadCache executes local load in caller thread

(cherry picked from commit e5f45fd)

(cherry picked from commit 2a6658e)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: ae9c6d603d04a46b4603d97d7db986bb33801228
Parents: 25e4706
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Fri Sep 22 15:33:37 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:33:37 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   | 10 +++++++++
 .../processors/task/GridTaskWorker.java         | 22 +++++++++++++++++++-
 .../resources/META-INF/classnames.properties    |  2 ++
 3 files changed, 33 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae9c6d60/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 ec79026..9992b0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Properties;
 import javax.net.ssl.HostnameVerifier;
 import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
@@ -706,6 +707,11 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_ENABLE_FORCIBLE_NODE_KILL = "IGNITE_ENABLE_FORCIBLE_NODE_KILL";
 
     /**
+     * If this property is set, then Ignite will use Async File IO factory by default.
+     */
+    public static final String IGNITE_USE_ASYNC_FILE_IO_FACTORY = "IGNITE_USE_ASYNC_FILE_IO_FACTORY";
+
+    /**
      * Tasks stealing will be started if tasks queue size per data-streamer thread exceeds this threshold.
      * <p>
      * Default value is {@code 4}.
@@ -721,6 +727,10 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_WAL_LOG_TX_RECORDS = "IGNITE_WAL_LOG_TX_RECORDS";
 
+    /** If this property is set, {@link PersistentStoreConfiguration#writeThrottlingEnabled} will be overridden to true
+     * independent of initial value in configuration. */
+    public static final String IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED = "IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae9c6d60/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 74fe57d..b94a427 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -574,7 +574,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         if (F.isEmpty(jobs))
             return;
 
-        Collection<GridJobResultImpl> jobResList = new ArrayList<>(jobs.size());
+        List<GridJobResultImpl> jobResList = new ArrayList<>(jobs.size());
 
         Collection<ComputeJobSibling> sibs = new ArrayList<>(jobs.size());
 
@@ -632,6 +632,26 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         // Set mapped flag.
         ses.onMapped();
 
+        // Move local jobs to the end of the list, because
+        // they will be invoked in current thread that will hold other
+        // jobs.
+        int jobResSize = jobResList.size();
+
+        if (jobResSize > 1) {
+            UUID locId = ctx.discovery().localNode().id();
+
+            for (int i = 0; i < jobResSize; i++) {
+                UUID jobNodeId = jobResList.get(i).getNode().id();
+
+                if (jobNodeId.equals(locId) && i < jobResSize - 1) {
+                    Collections.swap(jobResList, i, jobResSize - 1);
+
+                    jobResSize--;
+                    i--;
+                }
+            }
+        }
+
         // Send out all remote mappedJobs.
         for (GridJobResultImpl res : jobResList) {
             evtLsnr.onJobSend(this, res.getSibling());

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae9c6d60/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index e64363d..fd60bd4 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -2098,3 +2098,5 @@ org.apache.ignite.transactions.TransactionTimeoutException
 org.apache.ignite.util.AttributeNodeFilter
 org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO
 org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory
+org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO
+org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory


[42/50] [abbrv] ignite git commit: Fixed NPE in case of null value for indexing.

Posted by yz...@apache.org.
Fixed NPE in case of null value for indexing.

(cherry picked from commit 6cf3b13)

(cherry picked from commit dd9804b)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 25e4706501c1c0908df6d4cd070f84490c1ef6c4
Parents: 2310417
Author: mcherkasov <mc...@gridgain.com>
Authored: Thu Aug 31 12:11:01 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:29:38 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/database/InlineIndexHelper.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/25e47065/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
index 1789ac8..aa53a35 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
@@ -383,6 +383,9 @@ public class InlineIndexHelper {
         if (type == Value.NULL)
             return Integer.MIN_VALUE;
 
+        if (v == ValueNull.INSTANCE)
+            return fixSort(1, sortType());
+
         if (this.type != type)
             throw new UnsupportedOperationException("Invalid fast index type: " + type);
 


[50/50] [abbrv] ignite git commit: TCP communication skip local node loopback address

Posted by yz...@apache.org.
TCP communication skip local node loopback address


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: cc87a6215ed3d91948f6d0fa04b104a1e33a40be
Parents: d9a2786
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Oct 13 20:41:56 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Oct 13 20:41:56 2017 +0300

----------------------------------------------------------------------
 .../spi/communication/tcp/TcpCommunicationSpi.java   | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cc87a621/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 1dd48d1..15faeb3 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -2987,6 +2987,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
         if (isExtAddrsExist)
             addrs.addAll(extAddrs);
 
+        if (log.isDebugEnabled())
+            log.debug("Addresses resolved from attributes [rmtNode=" + node.id() + ", addrs=" + addrs +
+                ", isRmtAddrsExist=" + isRmtAddrsExist + ']');
+
         Set<InetAddress> allInetAddrs = U.newHashSet(addrs.size());
 
         for (InetSocketAddress addr : addrs) {
@@ -3015,7 +3019,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
         }
 
         if (log.isDebugEnabled())
-            log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs.toString() + ']');
+            log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs + ']');
 
         boolean conn = false;
         GridCommunicationClient client = null;
@@ -3024,6 +3028,15 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
         int connectAttempts = 1;
 
         for (InetSocketAddress addr : addrs) {
+            if (addr.getAddress().isLoopbackAddress() && addr.getPort() == boundTcpPort) {
+                if (log.isDebugEnabled())
+                    log.debug("Skipping local address [addr=" + addr +
+                        ", locAddrs=" + node.attribute(createSpiAttributeName(ATTR_ADDRS)) +
+                        ", node=" + node + ']');
+
+                continue;
+            }
+
             long connTimeout0 = connTimeout;
 
             int attempt = 1;


[46/50] [abbrv] ignite git commit: IGNITE-6063 InlineIdexHelperTest fails - Fixes #2444.

Posted by yz...@apache.org.
IGNITE-6063 InlineIdexHelperTest fails - Fixes #2444.

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

(cherry picked from commit 09d255e)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: a318c4a17b62a5876dd07b3cdcdb025b3ff03662
Parents: 6452201
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Tue Aug 15 14:46:32 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Sep 25 12:40:24 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/database/InlineIndexHelperTest.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a318c4a1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
index fc06502..6828218 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -495,7 +495,7 @@ public class InlineIndexHelperTest extends GridCommonAbstractTest {
         Value v1 = s1 == null ? ValueNull.INSTANCE : ValueString.get(s1);
         Value v2 = s2 == null ? ValueNull.INSTANCE : ValueString.get(s2);
 
-        int c = v1.compareTypeSafe(v2, CompareMode.getInstance(CompareMode.DEFAULT, 0));
+        int c = v1.compareTypeSafe(v2, CompareMode.getInstance(null, 0));
 
         return ha.canRelyOnCompare(c, v1, v2);
     }
@@ -505,7 +505,7 @@ public class InlineIndexHelperTest extends GridCommonAbstractTest {
         Value v1 = b1 == null ? ValueNull.INSTANCE : ValueBytes.get(b1);
         Value v2 = b2 == null ? ValueNull.INSTANCE : ValueBytes.get(b2);
 
-        int c = v1.compareTypeSafe(v2, CompareMode.getInstance(CompareMode.DEFAULT, 0));
+        int c = v1.compareTypeSafe(v2, CompareMode.getInstance(null, 0));
 
         return ha.canRelyOnCompare(c, v1, v2);
     }


[34/50] [abbrv] ignite git commit: IGNITE-6466 Fixed testGetForInitialWrite - Fixes #2721.

Posted by yz...@apache.org.
IGNITE-6466 Fixed testGetForInitialWrite - Fixes #2721.

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/98afbfda
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/98afbfda
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/98afbfda

Branch: refs/heads/ignite-2.1.5-p1
Commit: 98afbfdab47112cbae722a7317d1af39472e1f88
Parents: 1ccdc97
Author: dpavlov <dp...@gridgain.com>
Authored: Fri Sep 22 13:47:14 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Sep 22 13:49:44 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/persistence/tree/io/DataPageIO.java       | 6 +++---
 .../internal/processors/cache/persistence/tree/io/PageIO.java  | 2 +-
 .../IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java   | 4 +++-
 3 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/98afbfda/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
index e6876a1..628ff38 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
@@ -68,8 +68,8 @@ public class DataPageIO extends PageIO {
     /** */
     private static final int FIRST_ENTRY_OFF = INDIRECT_CNT_OFF + 1;
 
-    /** */
-    private static final int ITEMS_OFF = FIRST_ENTRY_OFF + 2;
+    /** Offset of items (internal page pointers) within data page */
+    public static final int ITEMS_OFF = FIRST_ENTRY_OFF + 2;
 
     /** */
     private static final int ITEM_SIZE = 2;
@@ -1445,7 +1445,7 @@ public class DataPageIO extends PageIO {
     /** {@inheritDoc} */
     @Override protected void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
         sb.a("DataPageIO [\n");
-        printPageLayout(addr, 0, sb);
+        printPageLayout(addr, pageSize, sb);
         sb.a("\n]");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/98afbfda/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
index 57339a1..b51a934 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
@@ -550,7 +550,7 @@ public abstract class PageIO {
 
     /**
      * @param addr Address.
-     * @param pageSize
+     * @param pageSize Page size.
      * @param sb Sb.
      */
     protected abstract void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException ;

http://git-wip-us.apache.org/repos/asf/ignite/blob/98afbfda/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
index 297e69f..2278981 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
@@ -235,8 +235,10 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                     try {
                         DataPageIO.VERSIONS.latest().initNewPage(pageAddr, fullId.pageId(), mem.pageSize());
 
-                        for (int i = PageIO.COMMON_HEADER_END; i < mem.pageSize(); i++)
+                        for (int i = PageIO.COMMON_HEADER_END + DataPageIO.ITEMS_OFF; i < mem.pageSize(); i++)
                             PageUtils.putByte(pageAddr, i, (byte)0xAB);
+
+                        PageIO.printPage(pageAddr, mem.pageSize());
                     }
                     finally {
                         mem.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, true);


[14/50] [abbrv] ignite git commit: IGNITE-6446 Fix permission transition hook and permissions for basic configuration screen. (cherry picked from commit 0f8a2bf)

Posted by yz...@apache.org.
IGNITE-6446 Fix permission transition hook and permissions for basic configuration screen.
(cherry picked from commit 0f8a2bf)


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 58ba04444d229476b502241940e474c25f0d895c
Parents: bd60ab1
Author: Ilya Borisov <kl...@gmail.com>
Authored: Thu Sep 21 10:25:19 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Sep 21 10:26:22 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/modules/states/configuration.state.js        | 1 +
 modules/web-console/frontend/app/modules/user/user.module.js  | 7 +++++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/58ba0444/modules/web-console/frontend/app/modules/states/configuration.state.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/modules/states/configuration.state.js
index d2c1410..8d05a45 100644
--- a/modules/web-console/frontend/app/modules/states/configuration.state.js
+++ b/modules/web-console/frontend/app/modules/states/configuration.state.js
@@ -77,6 +77,7 @@ angular.module('ignite-console.states.configuration', ['ui.router'])
             })
             .state('base.configuration.tabs.basic', {
                 url: '/basic',
+                permission: 'configuration',
                 template: '<page-configure-basic></page-configure-basic>',
                 tfMetaTags: {
                     title: 'Basic Configuration'

http://git-wip-us.apache.org/repos/asf/ignite/blob/58ba0444/modules/web-console/frontend/app/modules/user/user.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/user/user.module.js b/modules/web-console/frontend/app/modules/user/user.module.js
index 5cd8bc5..5ab0e0f 100644
--- a/modules/web-console/frontend/app/modules/user/user.module.js
+++ b/modules/web-console/frontend/app/modules/user/user.module.js
@@ -69,14 +69,14 @@ angular.module('ignite-console.user', [
         AclService.attachRole(role);
     });
 
-    $transitions.onEnter({}, (trans) => {
+    $transitions.onBefore({}, (trans) => {
         const $state = trans.router.stateService;
         const {name, permission} = trans.to();
 
         if (_.isEmpty(permission))
             return;
 
-        trans.injector().get('User').read()
+        return trans.injector().get('User').read()
             .then(() => {
                 if (AclService.can(permission)) {
                     Activities.post({action: $state.href(name, trans.params('to'))});
@@ -85,6 +85,9 @@ angular.module('ignite-console.user', [
                 }
 
                 return $state.target(trans.to().failState || '403');
+            })
+            .catch(() => {
+                return $state.target(trans.to().failState || '403');
             });
     });
 }]);


[19/50] [abbrv] ignite git commit: IGNITE-6448: Fixed a bug causing stale SQL statement to reside in cache after ALTER TABLE command. This closes #2702.

Posted by yz...@apache.org.
IGNITE-6448: Fixed a bug causing stale SQL statement to reside in cache after ALTER TABLE command. This closes #2702.


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: 099d33ceaee8bb8b3edbd1bee91a580572537e45
Parents: f5ff60e
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Sep 21 17:46:47 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Thu Sep 21 17:47:24 2017 +0300

----------------------------------------------------------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   3 +
 .../thin/JdbcThinSelectAfterAlterTable.java     | 173 +++++++++++++++++++
 .../processors/query/h2/IgniteH2Indexing.java   |   9 +
 .../processors/query/h2/opt/GridH2Table.java    |   2 +
 4 files changed, 187 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/099d33ce/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 0bd86b3..3dc00a4 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -54,6 +54,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinNoDefaultSchemaTest;
 import org.apache.ignite.jdbc.thin.JdbcThinPreparedStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinResultSetSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinSchemaCaseTest;
+import org.apache.ignite.jdbc.thin.JdbcThinSelectAfterAlterTable;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSelfTest;
 
@@ -147,6 +148,8 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         // New thin JDBC driver, full SQL tests
         suite.addTest(new TestSuite(JdbcThinComplexDmlDdlSelfTest.class));
 
+        suite.addTest(new TestSuite(JdbcThinSelectAfterAlterTable.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/099d33ce/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSelectAfterAlterTable.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSelectAfterAlterTable.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSelectAfterAlterTable.java
new file mode 100644
index 0000000..13b8f3e
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinSelectAfterAlterTable.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.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+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 org.jetbrains.annotations.NotNull;
+
+/**
+ * Base class for complex SQL tests based on JDBC driver.
+ */
+public class JdbcThinSelectAfterAlterTable extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Client connection port. */
+    private int cliPort = ClientConnectorConfiguration.DFLT_PORT;
+
+    /** JDBC connection. */
+    private Connection conn;
+
+    /** JDBC statement. */
+    private Statement stmt;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setClientConnectorConfiguration(new ClientConnectorConfiguration().setPort(cliPort++));
+
+        return cfg;
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     * @throws Exception In case of error.
+     */
+    private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(name);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1");
+
+        stmt = conn.createStatement();
+
+        stmt.executeUpdate("CREATE TABLE person (id LONG, name VARCHAR, city_id LONG, PRIMARY KEY (id, city_id))");
+        stmt.executeUpdate("INSERT INTO person (id, name, city_id) values (1, 'name_1', 11)");
+
+        stmt.executeQuery("select * from person");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stmt.close();
+        conn.close();
+
+        // Destroy all SQL caches after test.
+        for (String cacheName : grid(0).cacheNames()) {
+            DynamicCacheDescriptor cacheDesc = grid(0).context().cache().cacheDescriptor(cacheName);
+
+            if (cacheDesc != null && cacheDesc.sql())
+                grid(0).destroyCache0(cacheName, true);
+        }
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
+    public void testSelectAfterAlterTableSingleNode() throws Exception {
+        stmt.executeUpdate("alter table person add age int");
+
+        checkNewColumn(stmt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
+    public void testSelectAfterAlterTableMultiNode() throws Exception {
+        try (Connection conn2 = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1:"
+            + (ClientConnectorConfiguration.DFLT_PORT + 1))) {
+
+            try (Statement stmt2 = conn2.createStatement()) {
+                stmt2.executeUpdate("alter table person add age int");
+            }
+        }
+
+        checkNewColumn(stmt);
+    }
+
+    /**
+     * @param stmt Statement to check new column.
+     * @throws SQLException If failed.
+     */
+    public void checkNewColumn(Statement stmt) throws SQLException {
+        ResultSet rs = stmt.executeQuery("select * from person");
+
+        ResultSetMetaData meta = rs.getMetaData();
+
+        assertEquals(4, meta.getColumnCount());
+
+        boolean newColExists = false;
+
+        for (int i = 1; i <= meta.getColumnCount(); ++i) {
+            if ("age".equalsIgnoreCase(meta.getColumnName(i))) {
+                newColExists = true;
+
+                break;
+            }
+        }
+
+        assertTrue(newColExists);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/099d33ce/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 a07d48e..03e2391 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
@@ -729,6 +729,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         desc.table().addColumns(cols, ifColNotExists);
+
+        clearCachedQueries();
     }
 
     /**
@@ -2244,6 +2246,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
     }
 
+    /**
+     * Remove all cached queries from cached two-steps queries.
+     */
+    public void clearCachedQueries() {
+        twoStepCache.clear();
+    }
+
     /** {@inheritDoc} */
     @Override public IndexingQueryFilter backupFilter(@Nullable final AffinityTopologyVersion topVer,
         @Nullable final int[] parts) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/099d33ce/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 45bb9c7..549cbfb 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
@@ -999,6 +999,8 @@ public class GridH2Table extends TableBase {
             setColumns(newCols);
 
             desc.refreshMetadataFromTypeDescriptor();
+
+            setModified();
         }
         finally {
             unlock(true);