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

[01/39] ignite git commit: IGNITE-5163: Implemented infrastructure for the new JDBC driver. This closes #1912.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5075-cc-debug 58919c5b5 -> 0e069af54


http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java
deleted file mode 100644
index 3d21948..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.odbc;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.OdbcConfiguration;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * ODBC configuration validation tests.
- */
-public class OdbcProcessorValidationSelfTest extends GridCommonAbstractTest {
-    /** Node index generator. */
-    private static final AtomicInteger NODE_IDX_GEN = new AtomicInteger();
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * Ensure we can start with default configuration.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddressDefault() throws Exception {
-        check(new OdbcConfiguration(), true);
-    }
-
-    /**
-     * Test address where only host is provided.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddressHostOnly() throws Exception {
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1"), true);
-    }
-
-    /**
-     * Test address with both host and port.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddressHostAndPort() throws Exception {
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999"), true);
-
-        // Shouldn't fit into range.
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999"), false);
-    }
-
-    /**
-     * Test address with host and port range.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddressHostAndPortRange() throws Exception {
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..10000"), true);
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..10000"), true);
-
-        // Shouldn't fit into range.
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..10000"), false);
-    }
-
-    /**
-     * Test start with invalid host.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddressInvalidHost() throws Exception {
-        check(new OdbcConfiguration().setEndpointAddress("126.0.0.1"), false);
-    }
-
-    /**
-     * Test start with invalid address format.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAddressInvalidFormat() throws Exception {
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:"), false);
-
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:0"), false);
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:-1"), false);
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:111111"), false);
-
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999.."), false);
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..9998"), false);
-
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:a"), false);
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:a.."), false);
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:a..b"), false);
-
-        check(new OdbcConfiguration().setEndpointAddress(":9999"), false);
-        check(new OdbcConfiguration().setEndpointAddress(":9999..10000"), false);
-    }
-
-    /**
-     * Test connection parameters: sendBufferSize, receiveBufferSize, connectionTimeout.
-     *
-     * @throws Exception If failed.
-     */
-    public void testConnectionParams() throws Exception {
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
-            .setSocketSendBufferSize(4 * 1024), true);
-
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
-            .setSocketReceiveBufferSize(4 * 1024), true);
-
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
-            .setSocketSendBufferSize(-64 * 1024), false);
-
-        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
-            .setSocketReceiveBufferSize(-64 * 1024), false);
-    }
-
-    /**
-     * Test thread pool size.
-     *
-     * @throws Exception If failed.
-     */
-    public void testThreadPoolSize() throws Exception {
-        check(new OdbcConfiguration().setThreadPoolSize(0), false);
-        check(new OdbcConfiguration().setThreadPoolSize(-1), false);
-
-        check(new OdbcConfiguration().setThreadPoolSize(4), true);
-    }
-
-    /**
-     * Perform check.
-     *
-     * @param odbcCfg ODBC configuration.
-     * @param success Success flag. * @throws Exception If failed.
-     */
-    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    private void check(OdbcConfiguration odbcCfg, boolean success) throws Exception {
-        final IgniteConfiguration cfg = super.getConfiguration();
-
-        cfg.setIgniteInstanceName(OdbcProcessorValidationSelfTest.class.getName() + "-" + NODE_IDX_GEN.incrementAndGet());
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setOdbcConfiguration(odbcCfg);
-        cfg.setMarshaller(new BinaryMarshaller());
-
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-        spi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(spi);
-
-        if (success)
-            startGrid(cfg.getGridName(), cfg);
-        else {
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    startGrid(cfg.getGridName(), cfg);
-
-                    return null;
-                }
-            }, IgniteException.class, null);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessorValidationSelfTest.java
new file mode 100644
index 0000000..e433b6c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessorValidationSelfTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.odbc;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.OdbcConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * ODBC configuration validation tests.
+ */
+public class SqlListenerProcessorValidationSelfTest extends GridCommonAbstractTest {
+    /** Node index generator. */
+    private static final AtomicInteger NODE_IDX_GEN = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Ensure we can start with default configuration.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAddressDefault() throws Exception {
+        check(new OdbcConfiguration(), true);
+    }
+
+    /**
+     * Test address where only host is provided.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAddressHostOnly() throws Exception {
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1"), true);
+    }
+
+    /**
+     * Test address with both host and port.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAddressHostAndPort() throws Exception {
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999"), true);
+
+        // Shouldn't fit into range.
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999"), false);
+    }
+
+    /**
+     * Test address with host and port range.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAddressHostAndPortRange() throws Exception {
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..10000"), true);
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..10000"), true);
+
+        // Shouldn't fit into range.
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..10000"), false);
+    }
+
+    /**
+     * Test start with invalid host.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAddressInvalidHost() throws Exception {
+        check(new OdbcConfiguration().setEndpointAddress("126.0.0.1"), false);
+    }
+
+    /**
+     * Test start with invalid address format.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAddressInvalidFormat() throws Exception {
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:"), false);
+
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:0"), false);
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:-1"), false);
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:111111"), false);
+
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999.."), false);
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9999..9998"), false);
+
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:a"), false);
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:a.."), false);
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:a..b"), false);
+
+        check(new OdbcConfiguration().setEndpointAddress(":9999"), false);
+        check(new OdbcConfiguration().setEndpointAddress(":9999..10000"), false);
+    }
+
+    /**
+     * Test connection parameters: sendBufferSize, receiveBufferSize, connectionTimeout.
+     *
+     * @throws Exception If failed.
+     */
+    public void testConnectionParams() throws Exception {
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
+            .setSocketSendBufferSize(4 * 1024), true);
+
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
+            .setSocketReceiveBufferSize(4 * 1024), true);
+
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
+            .setSocketSendBufferSize(-64 * 1024), false);
+
+        check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000")
+            .setSocketReceiveBufferSize(-64 * 1024), false);
+    }
+
+    /**
+     * Test thread pool size.
+     *
+     * @throws Exception If failed.
+     */
+    public void testThreadPoolSize() throws Exception {
+        check(new OdbcConfiguration().setThreadPoolSize(0), false);
+        check(new OdbcConfiguration().setThreadPoolSize(-1), false);
+
+        check(new OdbcConfiguration().setThreadPoolSize(4), true);
+    }
+
+    /**
+     * Perform check.
+     *
+     * @param odbcCfg ODBC configuration.
+     * @param success Success flag. * @throws Exception If failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    private void check(OdbcConfiguration odbcCfg, boolean success) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration();
+
+        cfg.setIgniteInstanceName(SqlListenerProcessorValidationSelfTest.class.getName() + "-" +
+            NODE_IDX_GEN.incrementAndGet());
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setOdbcConfiguration(odbcCfg);
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+        spi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(spi);
+
+        if (success)
+            startGrid(cfg.getGridName(), cfg);
+        else {
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    startGrid(cfg.getGridName(), cfg);
+
+                    return null;
+                }
+            }, IgniteException.class, null);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 1a821a1..8559b97 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -344,7 +344,7 @@ public final class GridTestUtils {
             Throwable t = e;
 
             while (t != null) {
-                if (cls == t.getClass() && (msg == null || (t.getMessage() != null || t.getMessage().contains(msg)))) {
+                if (cls == t.getClass() && (msg == null || (t.getMessage() != null && t.getMessage().contains(msg)))) {
                     log.info("Caught expected exception: " + t.getMessage());
 
                     return t;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 0ec0f0d..ee69f24 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -52,7 +52,7 @@ import org.apache.ignite.internal.processors.database.BPlusTreeSelfTest;
 import org.apache.ignite.internal.processors.database.FreeListImplSelfTest;
 import org.apache.ignite.internal.processors.database.MetadataStorageSelfTest;
 import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest;
-import org.apache.ignite.internal.processors.odbc.OdbcProcessorValidationSelfTest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerProcessorValidationSelfTest;
 import org.apache.ignite.internal.processors.service.ClosureServiceClientsNodesTest;
 import org.apache.ignite.internal.product.GridProductVersionSelfTest;
 import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest;
@@ -142,7 +142,7 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
         suite.addTestSuite(IgniteLocalNodeMapBeforeStartTest.class);
-        suite.addTestSuite(OdbcProcessorValidationSelfTest.class);
+        suite.addTestSuite(SqlListenerProcessorValidationSelfTest.class);
         suite.addTestSuite(OdbcEscapeSequenceSelfTest.class);
 
         GridTestUtils.addTestIfNeeded(suite, DynamicProxySerializationMultiJvmSelfTest.class, ignoredTests);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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 aca214a..54c00fd 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
@@ -34,6 +34,14 @@ namespace ignite
 {
     namespace odbc
     {
+        struct ClientType
+        {
+            enum Type
+            {
+                ODBC = 0
+            };
+        };
+
         struct RequestType
         {
             enum Type
@@ -104,6 +112,8 @@ namespace ignite
                 writer.WriteInt16(version.GetMajor());
                 writer.WriteInt16(version.GetMinor());
                 writer.WriteInt16(version.GetMaintenance());
+                
+                writer.WriteInt8(ClientType::ODBC);
 
                 writer.WriteBool(distributedJoins);
                 writer.WriteBool(enforceJoinOrder);


[32/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 674e7dd23edaaff66b84d27794a006896647c408
Parents: f651e87
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 11:44:27 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 11:51:03 2017 +0300

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryEntry.java   | 23 +++++++++++++-------
 .../CacheContinuousQueryEventBuffer.java        | 18 ++++++++-------
 .../continuous/CacheContinuousQueryManager.java | 12 ++++++----
 .../CacheContinuousQueryPartitionRecovery.java  |  2 +-
 .../CacheContinuousQueryEventBufferTest.java    |  7 +++---
 ...eCacheContinuousQueryImmutableEntryTest.java |  4 ++--
 6 files changed, 40 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/674e7dd2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
index 28fdee3..3f463a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
@@ -128,6 +128,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
      * @param part Partition.
      * @param updateCntr Update partition counter.
      * @param topVer Topology version if applicable.
+     * @param flags Flags.
      */
     CacheContinuousQueryEntry(
         int cacheId,
@@ -138,7 +139,8 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
         boolean keepBinary,
         int part,
         long updateCntr,
-        @Nullable AffinityTopologyVersion topVer) {
+        @Nullable AffinityTopologyVersion topVer,
+        byte flags) {
         this.cacheId = cacheId;
         this.evtType = evtType;
         this.key = key;
@@ -147,9 +149,17 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
         this.part = part;
         this.updateCntr = updateCntr;
         this.topVer = topVer;
+        this.flags = flags;
 
         if (keepBinary)
-            flags |= KEEP_BINARY;
+            this.flags |= KEEP_BINARY;
+    }
+
+    /**
+     * @return Flags.
+     */
+    public byte flags() {
+        return flags;
     }
 
     /**
@@ -233,7 +243,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
         if (!isFiltered())
             return this;
 
-        CacheContinuousQueryEntry e = new CacheContinuousQueryEntry(
+        return new CacheContinuousQueryEntry(
             cacheId,
             null,
             null,
@@ -242,11 +252,8 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
             false,
             part,
             updateCntr,
-            topVer);
-
-        e.flags = flags;
-
-        return e;
+            topVer,
+            flags);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/674e7dd2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index a072240..afe34c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -363,7 +363,8 @@ public class CacheContinuousQueryEventBuffer {
                             e.isKeepBinary(),
                             e.partition(),
                             e.updateCounter(),
-                            e.topologyVersion());
+                            e.topologyVersion(),
+                            e.flags());
 
                         flushEntry.filteredCount(filtered);
 
@@ -399,13 +400,14 @@ public class CacheContinuousQueryEventBuffer {
         private CacheContinuousQueryEntry filteredEntry(long cntr, long filtered) {
             CacheContinuousQueryEntry e = new CacheContinuousQueryEntry(0,
                 null,
-                 null,
-                 null,
-                 null,
-                 false,
-                 part,
-                 cntr,
-                 topVer);
+                null,
+                null,
+                null,
+                false,
+                part,
+                cntr,
+                topVer,
+                (byte)0);
 
             e.markFiltered();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/674e7dd2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 7cbb1e1..1a655e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -193,7 +193,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                 lsnr.keepBinary(),
                 partId,
                 updCntr,
-                topVer);
+                topVer,
+                (byte)0);
 
             CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(
                 cctx.kernalContext().cache().jcache(cctx.name()), cctx, e0);
@@ -339,7 +340,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                 lsnr.keepBinary(),
                 partId,
                 updateCntr,
-                topVer);
+                topVer,
+                (byte)0);
 
             IgniteCacheProxy jcache = cctx.kernalContext().cache().jcacheProxy(cctx.name());
 
@@ -400,7 +402,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                     lsnr.keepBinary(),
                     e.partition(),
                     -1,
-                    null);
+                    null,
+                    (byte)0);
 
                 CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent(
                     cctx.kernalContext().cache().jcache(cctx.name()), cctx, e0);
@@ -703,7 +706,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                                     keepBinary,
                                     0,
                                     -1,
-                                    null);
+                                    null,
+                                    (byte)0);
 
                                 next = new CacheContinuousQueryEvent<>(
                                     cctx.kernalContext().cache().jcache(cctx.name()),

http://git-wip-us.apache.org/repos/asf/ignite/blob/674e7dd2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryPartitionRecovery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryPartitionRecovery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryPartitionRecovery.java
index 59252d2..e210c24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryPartitionRecovery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryPartitionRecovery.java
@@ -236,7 +236,7 @@ class CacheContinuousQueryPartitionRecovery {
                         lastFiredEvt = e.getKey();
 
                         if (e.getValue() != HOLE && !e.getValue().isFiltered())
-                            entries.add(new CacheContinuousQueryEvent<K, V>(cache, cctx, e.getValue()));
+                            entries.add(new CacheContinuousQueryEvent<K, V>(cache, cctx, pending));
 
                         iter.remove();
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/674e7dd2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBufferTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBufferTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBufferTest.java
index 4710593..382f166 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBufferTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBufferTest.java
@@ -120,8 +120,8 @@ public class CacheContinuousQueryEventBufferTest extends GridCommonAbstractTest
                 false,
                 0,
                 cntr,
-                null);
-
+                null,
+                (byte)0);
 
             entries.add(entry);
 
@@ -140,7 +140,8 @@ public class CacheContinuousQueryEventBufferTest extends GridCommonAbstractTest
                     false,
                     0,
                     cntr,
-                    null);
+                    null,
+                    (byte)0);
 
                 expEntry.filteredCount(filtered);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/674e7dd2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
index d230320..81a7515 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
@@ -35,7 +35,6 @@ import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
-import org.apache.ignite.internal.util.GridLongList;
 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;
@@ -138,7 +137,8 @@ public class IgniteCacheContinuousQueryImmutableEntryTest extends GridCommonAbst
             true,
             1,
             1L,
-            new AffinityTopologyVersion(1L));
+            new AffinityTopologyVersion(1L),
+            (byte)0);
 
         e0.markFiltered();
 


[25/39] ignite git commit: Clean up work directory for benchmarks.

Posted by sb...@apache.org.
Clean up work directory for benchmarks.


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 7325232b120b0c11e7ab3089958637ec99472eb9
Parents: 49fcd2cc
Author: oleg-ostanin <oo...@gridgain.com>
Authored: Thu May 25 19:32:09 2017 +0300
Committer: oleg-ostanin <oo...@gridgain.com>
Committed: Thu May 25 19:32:09 2017 +0300

----------------------------------------------------------------------
 .../yardstick/IgniteBenchmarkArguments.java     | 11 ++++
 .../org/apache/ignite/yardstick/IgniteNode.java |  5 ++
 .../apache/ignite/yardstick/io/FileUtils.java   | 64 ++++++++++++++++++++
 3 files changed, 80 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7325232b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index ea48bf3..74413f5 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -101,6 +101,10 @@ public class IgniteBenchmarkArguments {
     private boolean storeEnabled;
 
     /** */
+    @Parameter(names = {"-cwd", "--cleanWorkDirectory"}, description = "Clean Work Directory")
+    private boolean cleanWorkDirectory = false;
+
+    /** */
     @Parameter(names = {"-wb", "--writeBehind"}, description = "Enable or disable writeBehind for cache store")
     private boolean writeBehind;
 
@@ -422,6 +426,13 @@ public class IgniteBenchmarkArguments {
     }
 
     /**
+     * @return Flag for cleaning working directory.
+     */
+    public boolean cleanWorkDirectory() {
+        return cleanWorkDirectory;
+    }
+
+    /**
      * @return Description.
      */
     public String description() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7325232b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index 029c0fd..6e25fc4 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -32,8 +32,10 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.yardstick.io.FileUtils;
 import org.springframework.beans.BeansException;
 import org.springframework.beans.factory.xml.XmlBeanDefinitionReader;
 import org.springframework.context.ApplicationContext;
@@ -83,6 +85,9 @@ public class IgniteNode implements BenchmarkServer {
 
         assert c != null;
 
+        if (args.cleanWorkDirectory())
+            FileUtils.cleanDirectory(U.workDirectory(c.getWorkDirectory(), c.getIgniteHome()));
+
         ApplicationContext appCtx = tup.get2();
 
         assert appCtx != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7325232b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/FileUtils.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/FileUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/FileUtils.java
new file mode 100644
index 0000000..1bddc81
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/FileUtils.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.io;
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.LinkedList;
+
+/**
+ * Utility class for working with files.
+ */
+public class FileUtils {
+    /**
+     * Clean directory.
+     *
+     * @param path path to directory.
+     */
+    public static void cleanDirectory(String path) throws IOException {
+        LinkedList<Path> paths = new LinkedList<>();
+
+        appendOrRemove(paths, Files.newDirectoryStream(Paths.get(path)));
+
+        while (!paths.isEmpty()) {
+            if (Files.newDirectoryStream(paths.getLast()).iterator().hasNext())
+                appendOrRemove(paths, Files.newDirectoryStream(paths.getLast()));
+            else
+                Files.delete(paths.removeLast());
+        }
+    }
+
+    /**
+     * Add path to the stack if path is directory, otherwise delete it.
+     *
+     * @param paths Stack of paths.
+     * @param ds Stream of paths.
+     */
+    private static void appendOrRemove(LinkedList<Path> paths, DirectoryStream<Path> ds) throws IOException {
+        for (Path p : ds) {
+            if (Files.isDirectory(p))
+                paths.add(p);
+            else
+                Files.delete(p);
+        }
+    }
+}
+


[09/39] ignite git commit: IGNITE-5282: SQL: Correct top-bottom propagation of "keepBinary" flag. This closes #1993.

Posted by sb...@apache.org.
IGNITE-5282: SQL: Correct top-bottom propagation of "keepBinary" flag. This closes #1993.


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 33cb5e89c011cc0029898fd87e6b47e75809cf0c
Parents: eea7a3b
Author: devozerov <vo...@gridgain.com>
Authored: Wed May 24 00:02:26 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 24 00:02:26 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheProxy.java      | 15 +++++-----
 .../processors/query/GridQueryIndexing.java     | 11 +++++---
 .../processors/query/GridQueryProcessor.java    | 27 +++++++++---------
 .../query/h2/DmlStatementsProcessor.java        | 21 ++++++++------
 .../processors/query/h2/IgniteH2Indexing.java   | 29 +++++++++-----------
 .../cache/GridCacheCrossCacheQuerySelfTest.java | 12 ++++----
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  4 +--
 7 files changed, 62 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 40aafeb..a7d0d19 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -774,20 +774,19 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
             convertToBinary(qry);
 
-            final CacheOperationContext opCtxCall = ctx.operationContextPerCall();
+            CacheOperationContext opCtxCall = ctx.operationContextPerCall();
 
-            if (qry instanceof ContinuousQuery)
-                return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal(),
-                    opCtxCall != null && opCtxCall.isKeepBinary());
+            boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary();
 
-            if (qry instanceof SqlQuery) {
-                boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary();
+            if (qry instanceof ContinuousQuery)
+                return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal(), keepBinary);
 
+            if (qry instanceof SqlQuery)
                 return (QueryCursor<R>)ctx.kernalContext().query().querySql(ctx, (SqlQuery)qry, keepBinary);
-            }
 
             if (qry instanceof SqlFieldsQuery)
-                return (FieldsQueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry);
+                return (FieldsQueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry,
+                    keepBinary);
 
             if (qry instanceof ScanQuery)
                 return query((ScanQuery)qry, null, projection(qry.isLocal()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 1f5e701..4b9c0e2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -68,23 +68,25 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?,?> cctx, SqlQuery qry)
-        throws IgniteCheckedException;
+    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?,?> cctx, SqlQuery qry,
+        boolean keepBinary) throws IgniteCheckedException;
 
     /**
      * Parses SQL query into two step query and executes it.
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @param cancel Query cancel.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
     public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        GridQueryCancel cancel) throws IgniteCheckedException;
+        boolean keepBinary, GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Perform a MERGE statement using data streamer as receiver.
@@ -116,12 +118,13 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @param filter Cache name and key filter.
      * @param cancel Query cancel.
      * @return Cursor.
      */
     public FieldsQueryCursor<List<?>> queryLocalSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
+        boolean keepBinary, IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Executes text query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 ced78cf..a40c9e9 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
@@ -1651,10 +1651,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
     @SuppressWarnings("unchecked")
-    public FieldsQueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
+    public FieldsQueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry,
+        final boolean keepBinary) {
         checkxEnabled();
 
         if (qry.isReplicatedOnly() && qry.getPartitions() != null)
@@ -1677,7 +1679,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
                         GridQueryCancel cancel = new GridQueryCancel();
 
-                        final FieldsQueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry,
+                        final FieldsQueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry, keepBinary,
                             idx.backupFilter(requestTopVer.get(), qry.getPartitions()), cancel);
 
                         Iterable<List<?>> iterExec = new Iterable<List<?>>() {
@@ -1702,7 +1704,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             else {
                 clo = new IgniteOutClosureX<FieldsQueryCursor<List<?>>>() {
                     @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
-                        return idx.queryDistributedSqlFields(cctx, qry, null);
+                        return idx.queryDistributedSqlFields(cctx, qry, keepBinary, null);
                     }
                 };
             }
@@ -1767,16 +1769,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if ((qry.isReplicatedOnly() && cctx.isReplicatedAffinityNode()) || cctx.isLocal() || qry.isLocal())
             return queryLocalSql(cctx, qry, keepBinary);
 
-        return queryDistributedSql(cctx, qry);
+        return queryDistributedSql(cctx, qry, keepBinary);
     }
 
     /**
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryDistributedSql(final GridCacheContext<?,?> cctx,
-        final SqlQuery qry) {
+    private <K,V> QueryCursor<Cache.Entry<K,V>> queryDistributedSql(final GridCacheContext<?,?> cctx,
+        final SqlQuery qry, final boolean keepBinary) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
@@ -1786,7 +1789,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx,
                 new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
                     @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
-                        return idx.queryDistributedSql(cctx, qry);
+                        return idx.queryDistributedSql(cctx, qry, keepBinary);
                     }
                 }, true);
         }
@@ -1804,11 +1807,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(
-        final GridCacheContext<?, ?> cctx,
-        final SqlQuery qry,
-        final boolean keepBinary
-    ) {
+    private <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(final GridCacheContext<?, ?> cctx, final SqlQuery qry,
+        final boolean keepBinary) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
@@ -1827,7 +1827,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             qry.getArgs(),
                             cctx.name());
 
-                        return idx.queryLocalSql(cctx, qry, idx.backupFilter(requestTopVer.get(), qry.getPartitions()), keepBinary);
+                        return idx.queryLocalSql(cctx, qry, idx.backupFilter(requestTopVer.get(), qry.getPartitions()),
+                            keepBinary);
                     }
                 }, true);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index db7bfd6..e40c328 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -141,7 +141,8 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     private UpdateResult updateSqlFields(String cacheName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
-        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
+        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
         Object[] errKeys = null;
 
         long items = 0;
@@ -196,13 +197,13 @@ public class DmlStatementsProcessor {
     /**
      * @param cacheName Cache name.
      * @param stmt Prepared statement.
-     * @param fieldsQry Initial query.
+     * @param fieldsQry Initial query
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String cacheName, PreparedStatement stmt,
+    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String cacheName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
         UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, false, null, cancel);
 
@@ -218,14 +219,16 @@ public class DmlStatementsProcessor {
      * Execute DML statement on local cache.
      * @param cacheName Cache name.
      * @param stmt Prepared statement.
+     * @param fieldsQry Fields query.
      * @param filters Cache name and key filter.
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    GridQueryFieldsResult updateLocalSqlFields(String cacheName, PreparedStatement stmt,
-        SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
+    GridQueryFieldsResult updateSqlFieldsLocal(String cacheName, PreparedStatement stmt,
+        SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
         UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, true, filters, cancel);
 
         return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
@@ -266,8 +269,8 @@ public class DmlStatementsProcessor {
 
             final ArrayList<List<?>> data = new ArrayList<>(plan.rowsNum);
 
-            final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,
-                F.asList(args), null, false, 0, null);
+            final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry, F.asList(args),
+                null, false, 0, null);
 
             QueryCursorImpl<List<?>> stepCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
                 @Override public Iterator<List<?>> iterator() {
@@ -315,8 +318,10 @@ public class DmlStatementsProcessor {
 
     /**
      * Actually perform SQL DML operation locally.
+     *
      * @param cctx Cache context.
      * @param prepStmt Prepared statement for DML query.
+     * @param fieldsQry Fields query.
      * @param filters Cache name and key filter.
      * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
      * @return Pair [number of successfully processed items; keys that have failed to be processed]
@@ -351,7 +356,7 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(cctx, newFieldsQry, cancel);
+            cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(cctx, newFieldsQry, true, cancel);
         }
         else {
             final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 0ce905b..d3ee6ff 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
@@ -1027,8 +1027,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     @SuppressWarnings("unchecked")
     public GridQueryFieldsResult queryLocalSqlFields(final String cacheName, final String qry,
         @Nullable final Collection<Object> params, final IndexingQueryFilter filter, boolean enforceJoinOrder,
-        final int timeout, final GridQueryCancel cancel)
-        throws IgniteCheckedException {
+        final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException {
         final Connection conn = connectionForCache(cacheName);
 
         setupConnection(conn, false, enforceJoinOrder);
@@ -1046,7 +1045,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             fldsQry.setEnforceJoinOrder(enforceJoinOrder);
             fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
 
-            return dmlProc.updateLocalSqlFields(cacheName, stmt, fldsQry, filter, cancel);
+            return dmlProc.updateSqlFieldsLocal(cacheName, stmt, fldsQry, filter, cancel);
         }
         else if (DdlStatementsProcessor.isDdlStatement(p))
             throw new IgniteSQLException("DDL statements are supported for the whole cluster only",
@@ -1334,19 +1333,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
-        final SqlFieldsQuery qry, final IndexingQueryFilter filter, final GridQueryCancel cancel)
-        throws IgniteCheckedException {
+        final SqlFieldsQuery qry, final boolean keepBinary, final IndexingQueryFilter filter,
+        final GridQueryCancel cancel) throws IgniteCheckedException {
 
         if (cctx.config().getQueryParallelism() > 1) {
             qry.setDistributedJoins(true);
 
             assert qry.isLocal();
 
-            return queryDistributedSqlFields(cctx, qry, cancel);
+            return queryDistributedSqlFields(cctx, qry, keepBinary, cancel);
         }
         else {
-            final boolean keepBinary = cctx.keepBinary();
-
             final String cacheName = cctx.name();
             final String sql = qry.getSql();
             final Object[] args = qry.getArgs();
@@ -1379,7 +1376,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             assert qry.isLocal();
 
-            return queryDistributedSql(cctx, qry);
+            return queryDistributedSql(cctx, qry, keepBinary);
         }
         else {
             String cacheName = cctx.name();
@@ -1495,7 +1492,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?, ?> cctx, SqlQuery qry) {
+    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?, ?> cctx,
+        SqlQuery qry, boolean keepBinary) {
         String type = qry.getType();
         String cacheName = cctx.name();
 
@@ -1525,7 +1523,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (qry.getTimeout() > 0)
             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
 
-        final QueryCursor<List<?>> res = queryDistributedSqlFields(cctx, fqry, null);
+        final QueryCursor<List<?>> res = queryDistributedSqlFields(cctx, fqry, keepBinary, null);
 
         final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
             @Override public Iterator<Cache.Entry<K, V>> iterator() {
@@ -1567,7 +1565,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        GridQueryCancel cancel) {
+        boolean keepBinary, GridQueryCancel cancel) {
         final String cacheName = cctx.name();
         final String sqlQry = qry.getSql();
 
@@ -1657,7 +1655,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 if (twoStepQry == null) {
                     if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                         try {
-                            return dmlProc.updateSqlFieldsTwoStep(cctx.name(), stmt, qry, cancel);
+                            return dmlProc.updateSqlFieldsDistributed(cctx.name(), stmt, qry, cancel);
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DML statement [stmt=" + sqlQry +
@@ -1727,9 +1725,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             cancel = new GridQueryCancel();
 
         QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-            runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder, qry.getTimeout(), cancel,
-                    qry.getArgs(), qry.getPartitions()),
-            cancel);
+            runQueryTwoStep(cctx, twoStepQry, keepBinary, enforceJoinOrder, qry.getTimeout(), cancel,
+                qry.getArgs(), qry.getPartitions()), cancel);
 
         cursor.fieldsMeta(meta);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
index 42a71cd..6344cba 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
@@ -127,7 +127,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
         SqlFieldsQuery qry = new SqlFieldsQuery("select f.productId, p.name, f.price " +
             "from FactPurchase f, \"replicated-prod\".DimProduct p where p.id = f.productId ");
 
-        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             set1.add((Integer)o.get(0));
@@ -141,7 +141,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
 
         qry = new SqlFieldsQuery("select productId from FactPurchase group by productId");
 
-        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             assertTrue(set0.add((Integer) o.get(0)));
@@ -160,7 +160,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
             "where p.id = f.productId " +
             "group by f.productId, p.name");
 
-        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             assertTrue(names.add((String)o.get(0)));
@@ -177,7 +177,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
             "group by f.productId, p.name " +
             "having s >= 15");
 
-        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             assertTrue(i(o, 1) >= 15);
@@ -190,7 +190,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
         qry = new SqlFieldsQuery("select top 3 distinct productId " +
             "from FactPurchase f order by productId desc ");
 
-        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             assertEquals(top--, o.get(0));
@@ -203,7 +203,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
         qry = new SqlFieldsQuery("select distinct productId " +
             "from FactPurchase f order by productId desc limit 2 offset 1");
 
-        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             assertEquals(top--, o.get(0));

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 7ba7d56..97db2c4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -468,8 +468,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
                 time = now;
                 range *= 3;
 
-                GridQueryFieldsResult res = spi.queryLocalSqlFields("A", sql, Arrays.<Object>asList(1, range), null,
-                    false, 0, null);
+                GridQueryFieldsResult res = spi.queryLocalSqlFields("A", sql, Arrays.<Object>asList(1, range),
+                    null, false, 0, null);
 
                 assert res.iterator().hasNext();
 


[07/39] ignite git commit: Fixed a misprint in the non collocated join example

Posted by sb...@apache.org.
Fixed a misprint in the non collocated join example


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: b8875870257b8dc867283368067eb4387d7d9859
Parents: 794e1d3
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue May 23 13:45:34 2017 -0700
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue May 23 13:45:34 2017 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/examples/datagrid/CacheQueryExample.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b8875870/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java
index aa52cc1..4a6fc1b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java
@@ -222,7 +222,7 @@ public class CacheQueryExample {
             "where Person.orgId = org.id " +
             "and lower(org.name) = lower(?)";
 
-        SqlQuery qry = new SqlQuery<AffinityKey<Long>, Person>(Person.class, joinSql).
+        SqlQuery qry = new SqlQuery<Long, Person>(Person.class, joinSql).
             setArgs("ApacheIgnite");
 
         // Enable distributed joins for query.


[04/39] ignite git commit: Results printout for IO latency test and new metrics

Posted by sb...@apache.org.
Results printout for IO latency test and new metrics

(cherry picked)


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 018b25b29c3c491db7e44963e8c79677d77ceb23
Parents: 6f1dc3a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue May 23 17:39:37 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue May 23 17:44:33 2017 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 380 +++++++++++++++----
 .../communication/IgniteIoTestMessage.java      | 362 +++++++++++++++++-
 2 files changed, 672 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/018b25b2/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 147f94d..68bfd07 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
@@ -18,9 +18,11 @@
 package org.apache.ignite.internal.managers.communication;
 
 import java.io.Serializable;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -64,12 +66,12 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
 import org.apache.ignite.internal.processors.pool.PoolProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
-import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
@@ -341,10 +343,15 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                 IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg;
 
+                msg0.senderNodeId(nodeId);
+
                 if (msg0.request()) {
                     IgniteIoTestMessage res = new IgniteIoTestMessage(msg0.id(), false, null);
 
                     res.flags(msg0.flags());
+                    res.onRequestProcessed();
+
+                    res.copyDataFromRequest(msg0);
 
                     try {
                         sendToGridTopic(node, GridTopic.TOPIC_IO_TEST, res, GridIoPolicy.SYSTEM_POOL);
@@ -356,10 +363,12 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 else {
                     IoTestFuture fut = ioTestMap().get(msg0.id());
 
+                    msg0.onResponseProcessed();
+
                     if (fut == null)
                         U.warn(log, "Failed to find IO test future [msg=" + msg0 + ']');
                     else
-                        fut.onResponse();
+                        fut.onResponse(msg0);
                 }
             }
         });
@@ -404,7 +413,11 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @param procFromNioThread If {@code true} message is processed from NIO thread.
      * @return Response future.
      */
-    public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+    public IgniteInternalFuture<List<IgniteIoTestMessage>> sendIoTest(
+        ClusterNode node,
+        byte[] payload,
+        boolean procFromNioThread
+    ) {
         long id = ioTestId.getAndIncrement();
 
         IoTestFuture fut = new IoTestFuture(id, 1);
@@ -445,7 +458,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @param warmup Warmup duration in milliseconds.
      * @param duration Test duration in milliseconds.
      * @param threads Thread count.
-     * @param maxLatency Max latency in nanoseconds.
+     * @param latencyLimit Max latency in nanoseconds.
      * @param rangesCnt Ranges count in resulting histogram.
      * @param payLoadSize Payload size in bytes.
      * @param procFromNioThread {@code True} to process requests in NIO threads.
@@ -455,7 +468,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         final long warmup,
         final long duration,
         final int threads,
-        final long maxLatency,
+        final long latencyLimit,
         final int rangesCnt,
         final int payLoadSize,
         final boolean procFromNioThread,
@@ -469,8 +482,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         final LongAdder8 cnt = new LongAdder8();
         final long sleepDuration = 5000;
         final byte[] payLoad = new byte[payLoadSize];
-        final Map<UUID, long[]>[] res = new Map[threads];
-        final ConcurrentMap<UUID, GridAtomicLong> maxLatencies = new ConcurrentHashMap8<>();
+        final Map<UUID, IoTestThreadLocalNodeResults>[] res = new Map[threads];
 
         boolean failed = true;
 
@@ -489,7 +501,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                                 "[warmup=" + warmup +
                                 ", duration=" + duration +
                                 ", threads=" + threads +
-                                ", maxLatency=" + maxLatency +
+                                ", latencyLimit=" + latencyLimit +
                                 ", rangesCnt=" + rangesCnt +
                                 ", payLoadSize=" + payLoadSize +
                                 ", procFromNioThreads=" + procFromNioThread + ']'
@@ -529,22 +541,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                         // At this point all threads have finished the test and
                         // stored data to the resulting array of maps.
                         // Need to iterate it over and sum values for all threads.
-                        Map<UUID, long[]> res0 = new HashMap<>();
-
-                        for (Map<UUID, long[]> r : res) {
-                            for (Entry<UUID, long[]> e : r.entrySet()) {
-                                long[] r0 = res0.get(e.getKey());
-
-                                if (r0 == null)
-                                    res0.put(e.getKey(), e.getValue());
-                                else {
-                                    for (int i = 0; i < rangesCnt + 1; i++)
-                                        r0[i] += e.getValue()[i];
-                                }
-                            }
-                        }
-
-                        printIoTestResults(maxLatency / (1000 * rangesCnt), res0, maxLatencies);
+                        printIoTestResults(res);
                     }
                     catch (InterruptedException | BrokenBarrierException e) {
                         U.error(log, "IO test failed.", e);
@@ -566,7 +563,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                         boolean failed = true;
                         ThreadLocalRandom rnd = ThreadLocalRandom.current();
                         int size = nodes.size();
-                        Map<UUID, long[]> res0 = res[i0];
+                        Map<UUID, IoTestThreadLocalNodeResults> res0 = res[i0];
 
                         try {
                             boolean warmupFinished0 = false;
@@ -582,38 +579,22 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                                 ClusterNode node = nodes.get(rnd.nextInt(size));
 
-                                long start = System.nanoTime();
-
-                                sendIoTest(node, payLoad, procFromNioThread).get();
-
-                                long latency = System.nanoTime() - start;
+                                List<IgniteIoTestMessage> msgs = sendIoTest(node, payLoad, procFromNioThread).get();
 
                                 cnt.increment();
 
-                                long[] latencies = res0.get(node.id());
-
-                                if (latencies == null)
-                                    res0.put(node.id(), latencies = new long[rangesCnt + 1]);
-
-                                if (latency >= maxLatency) {
-                                    latencies[rangesCnt]++; // Timed out.
+                                for (IgniteIoTestMessage msg : msgs) {
+                                    UUID nodeId = msg.senderNodeId();
 
-                                    GridAtomicLong maxLatency = maxLatencies.get(node.id());
+                                    assert nodeId != null;
 
-                                    if (maxLatency == null) {
-                                        GridAtomicLong old = maxLatencies.putIfAbsent(node.id(),
-                                            maxLatency = new GridAtomicLong());
+                                    IoTestThreadLocalNodeResults nodeRes = res0.get(nodeId);
 
-                                        if (old != null)
-                                            maxLatency = old;
-                                    }
+                                    if (nodeRes == null)
+                                        res0.put(nodeId,
+                                            nodeRes = new IoTestThreadLocalNodeResults(rangesCnt, latencyLimit));
 
-                                    maxLatency.setIfGreater(latency);
-                                }
-                                else {
-                                    int idx = (int)Math.floor((1.0 * latency) / ((1.0 * maxLatency) / rangesCnt));
-
-                                    latencies[idx]++;
+                                    nodeRes.onResult(msg);
                                 }
                             }
 
@@ -641,30 +622,44 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     }
 
     /**
-     * @param binLatencyMcs Bin latency in microseconds.
-     * @param res Resulting map.
-     * @param maxLatencies Max latency for each node.
+     * @param rawRes Resulting map.
      */
     private void printIoTestResults(
-        long binLatencyMcs,
-        Map<UUID, long[]> res,
-        ConcurrentMap<UUID, GridAtomicLong> maxLatencies
+        Map<UUID, IoTestThreadLocalNodeResults>[] rawRes
     ) {
+        Map<UUID, IoTestNodeResults> res = new HashMap<>();
+
+        for (Map<UUID, IoTestThreadLocalNodeResults> r : rawRes) {
+            for (Entry<UUID, IoTestThreadLocalNodeResults> e : r.entrySet()) {
+                IoTestNodeResults r0 = res.get(e.getKey());
+
+                if (r0 == null)
+                    res.put(e.getKey(), r0 = new IoTestNodeResults());
+
+                r0.add(e.getValue());
+            }
+        }
+
+        SimpleDateFormat dateFmt = new SimpleDateFormat("HH:mm:ss,SSS");
+
         StringBuilder b = new StringBuilder(U.nl())
-            .append("IO test results (round-trip count per each latency bin) " +
-                "[binLatency=" + binLatencyMcs + "mcs]")
+            .append("IO test results (round-trip count per each latency bin).")
             .append(U.nl());
 
-        for (Entry<UUID, long[]> e : res.entrySet()) {
+        for (Entry<UUID, IoTestNodeResults> e : res.entrySet()) {
             ClusterNode node = ctx.discovery().node(e.getKey());
 
+            long binLatencyMcs = e.getValue().binLatencyMcs();
+
             b.append("Node ID: ").append(e.getKey()).append(" (addrs=")
-                .append(node != null ? node.addresses().toString() : "n/a").append(')').append(U.nl());
+                .append(node != null ? node.addresses().toString() : "n/a")
+                .append(", binLatency=").append(binLatencyMcs).append("mcs")
+                .append(')').append(U.nl());
 
             b.append("Latency bin, mcs | Count exclusive | Percentage exclusive | " +
                 "Count inclusive | Percentage inclusive ").append(U.nl());
 
-            long[] nodeRes = e.getValue();
+            long[] nodeRes = e.getValue().resLatency;
 
             long sum = 0;
 
@@ -688,15 +683,49 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                         curSum, (100.0 * curSum) / sum));
             }
 
-            GridAtomicLong maxLatency = maxLatencies.get(e.getKey());
+            b.append(U.nl()).append("Total latency (ns): ").append(U.nl())
+                .append(String.format("%15d", e.getValue().totalLatency)).append(U.nl());
+
+            b.append(U.nl()).append("Max latencies (ns):").append(U.nl());
+            format(b, e.getValue().maxLatency, dateFmt);
+
+            b.append(U.nl()).append("Max request send queue times (ns):").append(U.nl());
+            format(b, e.getValue().maxReqSendQueueTime, dateFmt);
+
+            b.append(U.nl()).append("Max request receive queue times (ns):").append(U.nl());
+            format(b, e.getValue().maxReqRcvQueueTime, dateFmt);
+
+            b.append(U.nl()).append("Max response send queue times (ns):").append(U.nl());
+            format(b, e.getValue().maxResSendQueueTime, dateFmt);
+
+            b.append(U.nl()).append("Max response receive queue times (ns):").append(U.nl());
+            format(b, e.getValue().maxResRcvQueueTime, dateFmt);
 
-            b.append("Max latency (ns): ").append(maxLatency != null ? maxLatency.get() : -1).append(U.nl());
+            b.append(U.nl()).append("Max request wire times (millis):").append(U.nl());
+            format(b, e.getValue().maxReqWireTimeMillis, dateFmt);
+
+            b.append(U.nl()).append("Max response wire times (millis):").append(U.nl());
+            format(b, e.getValue().maxResWireTimeMillis, dateFmt);
+
+            b.append(U.nl());
         }
 
         if (log.isInfoEnabled())
             log.info(b.toString());
     }
 
+    /**
+     * @param b Builder.
+     * @param pairs Pairs to format.
+     * @param dateFmt Formatter.
+     */
+    private void format(StringBuilder b, Collection<IgnitePair<Long>> pairs, SimpleDateFormat dateFmt) {
+        for (IgnitePair<Long> p : pairs) {
+            b.append(String.format("%15d", p.get1())).append(" ")
+                .append(dateFmt.format(new Date(p.get2()))).append(U.nl());
+        }
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings({"deprecation", "SynchronizationOnLocalVariableOrMethodParameter"})
     @Override public void onKernalStart0() throws IgniteCheckedException {
@@ -2857,12 +2886,15 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     /**
      *
      */
-    private class IoTestFuture extends GridFutureAdapter<Object> {
+    private class IoTestFuture extends GridFutureAdapter<List<IgniteIoTestMessage>> {
         /** */
         private final long id;
 
         /** */
-        private int cntr;
+        private final int cntr;
+
+        /** */
+        private final List<IgniteIoTestMessage> ress;
 
         /**
          * @param id ID.
@@ -2873,24 +2905,28 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
             this.id = id;
             this.cntr = cntr;
+
+            ress = new ArrayList<>(cntr);
         }
 
         /**
          *
          */
-        void onResponse() {
+        void onResponse(IgniteIoTestMessage res) {
             boolean complete;
 
             synchronized (this) {
-                complete = --cntr == 0;
+                ress.add(res);
+
+                complete = cntr == ress.size();
             }
 
             if (complete)
-                onDone();
+                onDone(ress);
         }
 
         /** {@inheritDoc} */
-        @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
+        @Override public boolean onDone(List<IgniteIoTestMessage> res, @Nullable Throwable err) {
             if (super.onDone(res, err)) {
                 ioTestMap().remove(id);
 
@@ -2905,4 +2941,210 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             return S.toString(IoTestFuture.class, this);
         }
     }
+
+    /**
+     *
+     */
+    private static class IoTestThreadLocalNodeResults {
+        /** */
+        private final long[] resLatency;
+
+        /** */
+        private final int rangesCnt;
+
+        /** */
+        private long totalLatency;
+
+        /** */
+        private long maxLatency;
+
+        /** */
+        private long maxLatencyTs;
+
+        /** */
+        private long maxReqSendQueueTime;
+
+        /** */
+        private long maxReqSendQueueTimeTs;
+
+        /** */
+        private long maxReqRcvQueueTime;
+
+        /** */
+        private long maxReqRcvQueueTimeTs;
+
+        /** */
+        private long maxResSendQueueTime;
+
+        /** */
+        private long maxResSendQueueTimeTs;
+
+        /** */
+        private long maxResRcvQueueTime;
+
+        /** */
+        private long maxResRcvQueueTimeTs;
+
+        /** */
+        private long maxReqWireTimeMillis;
+
+        /** */
+        private long maxReqWireTimeTs;
+
+        /** */
+        private long maxResWireTimeMillis;
+
+        /** */
+        private long maxResWireTimeTs;
+
+        /** */
+        private final long latencyLimit;
+
+        /**
+         * @param rangesCnt Ranges count.
+         * @param latencyLimit
+         */
+        public IoTestThreadLocalNodeResults(int rangesCnt, long latencyLimit) {
+            this.rangesCnt = rangesCnt;
+            this.latencyLimit = latencyLimit;
+
+            resLatency = new long[rangesCnt + 1];
+        }
+
+        /**
+         * @param msg
+         */
+        public void onResult(IgniteIoTestMessage msg) {
+            long now = System.currentTimeMillis();
+
+            long latency = msg.responseProcessedTs() - msg.requestCreateTs();
+
+            int idx = latency >= latencyLimit ?
+                rangesCnt /* Timed out. */ :
+                (int)Math.floor((1.0 * latency) / ((1.0 * latencyLimit) / rangesCnt));
+
+            resLatency[idx]++;
+
+            totalLatency += latency;
+
+            if (maxLatency < latency) {
+                maxLatency = latency;
+                maxLatencyTs = now;
+            }
+
+            long reqSndQueueTime = msg.requestSendTs() - msg.requestCreateTs();
+
+            if (maxReqSendQueueTime < reqSndQueueTime) {
+                maxReqSendQueueTime = reqSndQueueTime;
+                maxReqSendQueueTimeTs = now;
+            }
+
+            long reqRcvQueueTime = msg.requestProcessTs() - msg.requestReceiveTs();
+
+            if (maxReqRcvQueueTime < reqRcvQueueTime) {
+                maxReqRcvQueueTime = reqRcvQueueTime;
+                maxReqRcvQueueTimeTs = now;
+            }
+
+            long resSndQueueTime = msg.responseSendTs() - msg.requestProcessTs();
+
+            if (maxResSendQueueTime < resSndQueueTime) {
+                maxResSendQueueTime = resSndQueueTime;
+                maxResSendQueueTimeTs = now;
+            }
+
+            long resRcvQueueTime = msg.responseProcessedTs() - msg.responseReceiveTs();
+
+            if (maxResRcvQueueTime < resRcvQueueTime) {
+                maxResRcvQueueTime = resRcvQueueTime;
+                maxResRcvQueueTimeTs = now;
+            }
+
+            long reqWireTimeMillis = msg.requestReceivedTsMillis() - msg.requestSendTsMillis();
+
+            if (maxReqWireTimeMillis < reqWireTimeMillis) {
+                maxReqWireTimeMillis = reqWireTimeMillis;
+                maxReqWireTimeTs = now;
+            }
+
+            long resWireTimeMillis = msg.responseRecievedTsMillis() - msg.requestSendTsMillis();
+
+            if (maxResWireTimeMillis < resWireTimeMillis) {
+                maxResWireTimeMillis = resWireTimeMillis;
+                maxResWireTimeTs = now;
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class IoTestNodeResults {
+        /** */
+        private long latencyLimit;
+
+        /** */
+        private long[] resLatency;
+
+        /** */
+        private long totalLatency;
+
+        /** */
+        private Collection<IgnitePair<Long>> maxLatency = new ArrayList<>();
+
+        /** */
+        private Collection<IgnitePair<Long>> maxReqSendQueueTime = new ArrayList<>();
+
+        /** */
+        private Collection<IgnitePair<Long>> maxReqRcvQueueTime = new ArrayList<>();
+
+        /** */
+        private Collection<IgnitePair<Long>> maxResSendQueueTime = new ArrayList<>();
+
+        /** */
+        private Collection<IgnitePair<Long>> maxResRcvQueueTime = new ArrayList<>();
+
+        /** */
+        private Collection<IgnitePair<Long>> maxReqWireTimeMillis = new ArrayList<>();
+
+        /** */
+        private Collection<IgnitePair<Long>> maxResWireTimeMillis = new ArrayList<>();
+
+        /**
+         * @param res Node results to add.
+         */
+        public void add(IoTestThreadLocalNodeResults res) {
+            if (resLatency == null) {
+                resLatency = res.resLatency.clone();
+                latencyLimit = res.latencyLimit;
+            }
+            else {
+                assert latencyLimit == res.latencyLimit;
+                assert resLatency.length == res.resLatency.length;
+
+                for (int i = 0; i < resLatency.length; i++)
+                    resLatency[i] += res.resLatency[i];
+            }
+
+            totalLatency += res.totalLatency;
+
+            maxLatency.add(F.pair(res.maxLatency, res.maxLatencyTs));
+            maxReqSendQueueTime.add(F.pair(res.maxReqSendQueueTime, res.maxReqSendQueueTimeTs));
+            maxReqRcvQueueTime.add(F.pair(res.maxReqRcvQueueTime, res.maxReqRcvQueueTimeTs));
+            maxResSendQueueTime.add(F.pair(res.maxResSendQueueTime, res.maxResSendQueueTimeTs));
+            maxResRcvQueueTime.add(F.pair(res.maxResRcvQueueTime, res.maxResRcvQueueTimeTs));
+            maxReqWireTimeMillis.add(F.pair(res.maxReqWireTimeMillis, res.maxReqWireTimeTs));
+            maxResWireTimeMillis.add(F.pair(res.maxResWireTimeMillis, res.maxResWireTimeTs));
+        }
+
+        /**
+         * @return Bin latency in microseconds.
+         */
+        public long binLatencyMcs() {
+            if (resLatency == null)
+                throw new IllegalStateException();
+
+            return latencyLimit / (1000 * (resLatency.length - 1));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/018b25b2/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
index 0a41622..3e0fa76 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.managers.communication;
 
 import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -45,6 +47,43 @@ public class IgniteIoTestMessage implements Message {
     /** */
     private byte payload[];
 
+    /** */
+    private long reqCreateTs;
+
+    /** */
+    private long reqSndTs;
+
+    /** */
+    private long reqSndTsMillis;
+
+    /** */
+    private long reqRcvTs;
+
+    /** */
+    private long reqRcvTsMillis;
+
+    /** */
+    private long reqProcTs;
+
+    /** */
+    private long resSndTs;
+
+    /** */
+    private long resSndTsMillis;
+
+    /** */
+    private long resRcvTs;
+
+    /** */
+    private long resRcvTsMillis;
+
+    /** */
+    private long resProcTs;
+
+    /** */
+    @GridDirectTransient
+    private UUID sndNodeId;
+
     /**
      *
      */
@@ -61,6 +100,8 @@ public class IgniteIoTestMessage implements Message {
         this.id = id;
         this.req = req;
         this.payload = payload;
+
+        reqCreateTs = System.nanoTime();
     }
 
     /**
@@ -126,10 +167,173 @@ public class IgniteIoTestMessage implements Message {
         return id;
     }
 
+    /**
+     * @return Request create timestamp.
+     */
+    public long requestCreateTs() {
+        return reqCreateTs;
+    }
+
+    /**
+     * @return Request send timestamp.
+     */
+    public long requestSendTs() {
+        return reqSndTs;
+    }
+
+    /**
+     * @return Request receive timestamp.
+     */
+    public long requestReceiveTs() {
+        return reqRcvTs;
+    }
+
+    /**
+     * @return Request process started timestamp.
+     */
+    public long requestProcessTs() {
+        return reqProcTs;
+    }
+
+    /**
+     * @return Response send timestamp.
+     */
+    public long responseSendTs() {
+        return resSndTs;
+    }
+
+    /**
+     * @return Response receive timestamp.
+     */
+    public long responseReceiveTs() {
+        return resRcvTs;
+    }
+
+    /**
+     * @return Response process timestamp.
+     */
+    public long responseProcessTs() {
+        return resProcTs;
+    }
+
+    /**
+     * @return Request send timestamp (millis).
+     */
+    public long requestSendTsMillis() {
+        return reqSndTsMillis;
+    }
+
+    /**
+     * @return Request received timestamp (millis).
+     */
+    public long requestReceivedTsMillis() {
+        return reqRcvTsMillis;
+    }
+
+    /**
+     * @return Response send timestamp (millis).
+     */
+    public long responseSendTsMillis() {
+        return resSndTsMillis;
+    }
+
+    /**
+     * @return Response received timestamp (millis).
+     */
+    public long responseRecievedTsMillis() {
+        return resRcvTsMillis;
+    }
+
+    /**
+     * This method is called to initialize tracing variables.
+     * TODO: introduce direct message lifecycle API?
+     */
+    public void onAfterRead() {
+        if (req && reqRcvTs == 0) {
+            reqRcvTs = System.nanoTime();
+
+            reqRcvTsMillis = System.currentTimeMillis();
+        }
+
+        if (!req && resRcvTs == 0) {
+            resRcvTs = System.nanoTime();
+
+            resRcvTsMillis = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * This method is called to initialize tracing variables.
+     * TODO: introduce direct message lifecycle API?
+     */
+    public void onBeforeWrite() {
+        if (req && reqSndTs == 0) {
+            reqSndTs = System.nanoTime();
+
+            reqSndTsMillis = System.currentTimeMillis();
+        }
+
+        if (!req && resSndTs == 0) {
+            resSndTs = System.nanoTime();
+
+            resSndTsMillis = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     *
+     */
+    public void copyDataFromRequest(IgniteIoTestMessage req) {
+        reqCreateTs = req.reqCreateTs;
+
+        reqSndTs = req.reqSndTs;
+        reqSndTsMillis = req.reqSndTsMillis;
+
+        reqRcvTs = req.reqRcvTs;
+        reqRcvTsMillis = req.reqRcvTsMillis;
+    }
+
+    /**
+     *
+     */
+    public void onRequestProcessed() {
+        reqProcTs = System.nanoTime();
+    }
+
+    /**
+     *
+     */
+    public void onResponseProcessed() {
+        resProcTs = System.nanoTime();
+    }
+
+    /**
+     * @return Response processed timestamp.
+     */
+    public long responseProcessedTs() {
+        return resProcTs;
+    }
+
+    /**
+     * @return Sender node ID.
+     */
+    public UUID senderNodeId() {
+        return sndNodeId;
+    }
+
+    /**
+     * @param sndNodeId Sender node ID.
+     */
+    public void senderNodeId(UUID sndNodeId) {
+        this.sndNodeId = sndNodeId;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 
+        onBeforeWrite();
+
         if (!writer.isHeaderWritten()) {
             if (!writer.writeHeader(directType(), fieldsCount()))
                 return false;
@@ -162,6 +366,72 @@ public class IgniteIoTestMessage implements Message {
 
                 writer.incrementState();
 
+            case 4:
+                if (!writer.writeLong("reqCreateTs", reqCreateTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeLong("reqProcTs", reqProcTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeLong("reqRcvTs", reqRcvTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
+                if (!writer.writeLong("reqRcvTsMillis", reqRcvTsMillis))
+                    return false;
+
+                writer.incrementState();
+
+            case 8:
+                if (!writer.writeLong("reqSndTs", reqSndTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 9:
+                if (!writer.writeLong("reqSndTsMillis", reqSndTsMillis))
+                    return false;
+
+                writer.incrementState();
+
+            case 10:
+                if (!writer.writeLong("resProcTs", resProcTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 11:
+                if (!writer.writeLong("resRcvTs", resRcvTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 12:
+                if (!writer.writeLong("resRcvTsMillis", resRcvTsMillis))
+                    return false;
+
+                writer.incrementState();
+
+            case 13:
+                if (!writer.writeLong("resSndTs", resSndTs))
+                    return false;
+
+                writer.incrementState();
+
+            case 14:
+                if (!writer.writeLong("resSndTsMillis", resSndTsMillis))
+                    return false;
+
+                writer.incrementState();
+
         }
 
         return true;
@@ -207,8 +477,98 @@ public class IgniteIoTestMessage implements Message {
 
                 reader.incrementState();
 
+            case 4:
+                reqCreateTs = reader.readLong("reqCreateTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                reqProcTs = reader.readLong("reqProcTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
+                reqRcvTs = reader.readLong("reqRcvTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
+                reqRcvTsMillis = reader.readLong("reqRcvTsMillis");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 8:
+                reqSndTs = reader.readLong("reqSndTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 9:
+                reqSndTsMillis = reader.readLong("reqSndTsMillis");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 10:
+                resProcTs = reader.readLong("resProcTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 11:
+                resRcvTs = reader.readLong("resRcvTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 12:
+                resRcvTsMillis = reader.readLong("resRcvTsMillis");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 13:
+                resSndTs = reader.readLong("resSndTs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 14:
+                resSndTsMillis = reader.readLong("resSndTsMillis");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
         }
 
+        onAfterRead();
+
         return reader.afterMessageRead(IgniteIoTestMessage.class);
     }
 
@@ -219,7 +579,7 @@ public class IgniteIoTestMessage implements Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 4;
+        return 15;
     }
 
     /** {@inheritDoc} */


[06/39] ignite git commit: IGNITE-5281: Indexing: changed "space" to "cacheName". No more "spaces". This closes #1992.

Posted by sb...@apache.org.
IGNITE-5281: Indexing: changed "space" to "cacheName". No more "spaces". This closes #1992.

:quit
:quit
mdules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js~HEAD


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: cbf0b2a5873395b127e05b411136f2d30f449573
Parents: 018b25b
Author: devozerov <vo...@gridgain.com>
Authored: Tue May 23 20:37:55 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue May 23 20:38:49 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/jdbc2/JdbcConnection.java   |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  19 +-
 .../cache/query/GridCacheQueryManager.java      |  32 +--
 .../processors/query/GridQueryIndexing.java     |  85 +++---
 .../processors/query/GridQueryProcessor.java    | 153 +++++------
 .../processors/query/QueryIndexKey.java         |  20 +-
 .../query/QueryTypeDescriptorImpl.java          |  16 +-
 .../processors/query/QueryTypeIdKey.java        |  26 +-
 .../processors/query/QueryTypeNameKey.java      |  12 +-
 .../internal/processors/query/QueryUtils.java   |  14 +-
 .../schema/SchemaIndexCacheVisitorImpl.java     |  12 +-
 .../operation/SchemaAbstractOperation.java      |   2 +-
 .../spi/indexing/IndexingQueryFilter.java       |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |  17 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |   6 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   6 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |  14 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |   6 +-
 .../query/h2/DmlStatementsProcessor.java        |  51 ++--
 .../processors/query/h2/IgniteH2Indexing.java   | 258 +++++++++----------
 .../query/h2/database/H2PkHashIndex.java        |   4 +-
 .../query/h2/database/H2TreeIndex.java          |   4 +-
 .../query/h2/opt/GridH2IndexBase.java           |  10 +-
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |   2 +-
 .../query/h2/opt/GridLuceneIndex.java           |  15 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  30 +--
 .../DynamicIndexAbstractBasicSelfTest.java      |  18 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |   8 +-
 .../query/IgniteQueryDedicatedPoolTest.java     |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 136 +++++-----
 .../query/h2/sql/GridQueryParsingTest.java      |   2 +-
 33 files changed, 491 insertions(+), 508 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 8da385a..f6f79fb 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
@@ -47,7 +47,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
-import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteJdbcDriver;
@@ -708,7 +707,7 @@ public class JdbcConnection implements Connection {
     @Override public void setSchema(String schema) throws SQLException {
         assert ignite instanceof IgniteEx;
 
-        cacheName = ((IgniteEx)ignite).context().query().space(schema);
+        cacheName = ((IgniteEx)ignite).context().query().cacheName(schema);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
index a60cdbd..6d29604 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
@@ -79,14 +79,15 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
     /**
      * Writes key-value pair to index.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> void store(final String space, final K key, final V val, long expirationTime) throws IgniteCheckedException {
+    public <K, V> void store(final String cacheName, final K key, final V val, long expirationTime)
+        throws IgniteCheckedException {
         assert key != null;
         assert val != null;
         assert enabled();
@@ -98,7 +99,7 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
             if (log.isDebugEnabled())
                 log.debug("Storing key to cache query index [key=" + key + ", value=" + val + "]");
 
-            getSpi().store(space, key, val, expirationTime);
+            getSpi().store(cacheName, key, val, expirationTime);
         }
         finally {
             busyLock.leaveBusy();
@@ -106,12 +107,12 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings("unchecked")
-    public void remove(String space, Object key) throws IgniteCheckedException {
+    public void remove(String cacheName, Object key) throws IgniteCheckedException {
         assert key != null;
         assert enabled();
 
@@ -119,7 +120,7 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
             throw new IllegalStateException("Failed to remove from index (grid is stopping).");
 
         try {
-            getSpi().remove(space, key);
+            getSpi().remove(cacheName, key);
         }
         finally {
             busyLock.leaveBusy();
@@ -127,14 +128,14 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param params Parameters collection.
      * @param filters Filters.
      * @return Query result.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public IgniteSpiCloseableIterator<?> query(String space, Collection<Object> params, IndexingQueryFilter filters)
+    public IgniteSpiCloseableIterator<?> query(String cacheName, Collection<Object> params, IndexingQueryFilter filters)
         throws IgniteCheckedException {
         if (!enabled())
             throw new IgniteCheckedException("Indexing SPI is not configured.");
@@ -143,7 +144,7 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            final Iterator<?> res = getSpi().query(space, params, filters);
+            final Iterator<?> res = getSpi().query(cacheName, params, filters);
 
             if (res == null)
                 return new GridEmptyCloseableIterator<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 0a789ae..07545a5 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
@@ -173,7 +173,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     private GridQueryProcessor qryProc;
 
     /** */
-    private String space;
+    private String cacheName;
 
     /** */
     private int maxIterCnt;
@@ -221,9 +221,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         qryProc = cctx.kernalContext().query();
 
-        space = cctx.name();
+        cacheName = cctx.name();
 
-        enabled = qryProcEnabled || (isIndexingSpiEnabled() && !CU.isSystemCache(space));
+        enabled = qryProcEnabled || (isIndexingSpiEnabled() && !CU.isSystemCache(cacheName));
 
         maxIterCnt = ccfg.getMaxQueryIteratorsCount();
 
@@ -409,11 +409,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 Object val0 = unwrapIfNeeded(val, coctx);
 
-                cctx.kernalContext().indexing().store(space, key0, val0, expirationTime);
+                cctx.kernalContext().indexing().store(cacheName, key0, val0, expirationTime);
             }
 
             if(qryProcEnabled)
-                qryProc.store(space, key, partId, prevVal, prevVer, val, ver, expirationTime, link);
+                qryProc.store(cacheName, key, partId, prevVal, prevVer, val, ver, expirationTime, link);
         }
         finally {
             invalidateResultCache();
@@ -443,12 +443,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             if (isIndexingSpiEnabled()) {
                 Object key0 = unwrapIfNeeded(key, cctx.cacheObjectContext());
 
-                cctx.kernalContext().indexing().remove(space, key0);
+                cctx.kernalContext().indexing().remove(cacheName, key0);
             }
 
             // val may be null if we have no previous value. We should not call processor in this case.
             if(qryProcEnabled && val != null)
-                qryProc.remove(space, key, partId, val, ver);
+                qryProc.remove(cacheName, key, partId, val, ver);
         }
         finally {
             invalidateResultCache();
@@ -628,7 +628,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                             taskName));
                     }
 
-                    iter = qryProc.queryText(space, qry.clause(), qry.queryClassName(), filter(qry));
+                    iter = qryProc.queryText(cacheName, qry.clause(), qry.queryClassName(), filter(qry));
 
                     break;
 
@@ -739,7 +739,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         try {
             if (qry.type() == SPI) {
-                IgniteSpiCloseableIterator<?> iter = cctx.kernalContext().indexing().query(space, F.asList(args),
+                IgniteSpiCloseableIterator<?> iter = cctx.kernalContext().indexing().query(cacheName, F.asList(args),
                     filter(qry));
 
                 res.onDone(iter);
@@ -1907,7 +1907,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             // Remote nodes that have current cache.
             Collection<ClusterNode> nodes = F.view(cctx.discovery().remoteNodes(), new P1<ClusterNode>() {
                 @Override public boolean apply(ClusterNode n) {
-                    return cctx.kernalContext().discovery().cacheAffinityNode(n, space);
+                    return cctx.kernalContext().discovery().cacheAffinityNode(n, cacheName);
                 }
             });
 
@@ -1945,7 +1945,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             Collection<GridCacheSqlMetadata> col = new ArrayList<>(map.size());
 
             // Metadata for current cache must be first in list.
-            col.add(new CacheSqlMetadata(map.remove(space)));
+            col.add(new CacheSqlMetadata(map.remove(cacheName)));
 
             for (Collection<CacheSqlMetadata> metas : map.values())
                 col.add(new CacheSqlMetadata(metas));
@@ -1969,10 +1969,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             return null;
 
         return new IndexingQueryFilter() {
-            @Nullable @Override public IgniteBiPredicate<K, V> forSpace(final String spaceName) {
+            @Nullable @Override public IgniteBiPredicate<K, V> forCache(final String cacheName) {
                 final GridKernalContext ctx = cctx.kernalContext();
 
-                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(spaceName);
+                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(cacheName);
 
                 if (cache.context().isReplicated() || cache.configuration().getBackups() == 0)
                     return null;
@@ -2016,10 +2016,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * FOR TESTING ONLY
      *
-     * @return Indexing space for this query manager.
+     * @return Cache name for this query manager.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index bfa223e..1f5e701 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -89,14 +89,14 @@ public interface GridQueryIndexing {
     /**
      * Perform a MERGE statement using data streamer as receiver.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param params Query parameters.
      * @param streamer Data streamer to feed data to.
      * @return Query result.
      * @throws IgniteCheckedException If failed.
      */
-    public long streamUpdateQuery(final String spaceName, final String qry, @Nullable final Object[] params,
+    public long streamUpdateQuery(String cacheName, String qry, @Nullable Object[] params,
         IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException;
 
     /**
@@ -104,7 +104,7 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
@@ -116,7 +116,7 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @param cancel Query cancel.
      * @return Cursor.
      */
@@ -126,84 +126,84 @@ public interface GridQueryIndexing {
     /**
      * Executes text query.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param qry Text query.
      * @param typeName Type name.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @return Queried rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String spaceName, String qry,
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String cacheName, String qry,
         String typeName, IndexingQueryFilter filter) throws IgniteCheckedException;
 
     /**
      * Create new index locally.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @param idxDesc Index descriptor.
      * @param ifNotExists Ignore operation if index exists (instead of throwing an error).
      * @param cacheVisitor Cache visitor
      * @throws IgniteCheckedException if failed.
      */
-    public void dynamicIndexCreate(String spaceName, String tblName, QueryIndexDescriptorImpl idxDesc,
+    public void dynamicIndexCreate(String cacheName, String tblName, QueryIndexDescriptorImpl idxDesc,
         boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException;
 
     /**
-     * Remove index from the space.
+     * Remove index from the cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName cache name.
      * @param idxName Index name.
      * @param ifExists Ignore operation if index does not exist (instead of throwing an error).
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-    public void dynamicIndexDrop(String spaceName, String idxName, boolean ifExists)
+    public void dynamicIndexDrop(String cacheName, String idxName, boolean ifExists)
         throws IgniteCheckedException;
 
     /**
      * Registers cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param cctx Cache context.
      * @param ccfg Cache configuration.
      * @throws IgniteCheckedException If failed.
      */
-    public void registerCache(String spaceName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg)
+    public void registerCache(String cacheName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg)
         throws IgniteCheckedException;
 
     /**
      * Unregisters cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @throws IgniteCheckedException If failed to drop cache schema.
      */
-    public void unregisterCache(String spaceName) throws IgniteCheckedException;
+    public void unregisterCache(String cacheName) throws IgniteCheckedException;
 
     /**
      * Registers type if it was not known before or updates it otherwise.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param desc Type descriptor.
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if type was registered, {@code false} if for some reason it was rejected.
      */
-    public boolean registerType(String spaceName, GridQueryTypeDescriptor desc) throws IgniteCheckedException;
+    public boolean registerType(String cacheName, GridQueryTypeDescriptor desc) throws IgniteCheckedException;
 
     /**
      * Unregisters type and removes all corresponding data.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @throws IgniteCheckedException If failed.
      */
-    public void unregisterType(String spaceName, String typeName) throws IgniteCheckedException;
+    public void unregisterType(String cacheName, String typeName) throws IgniteCheckedException;
 
     /**
-     * Updates index. Note that key is unique for space, so if space contains multiple indexes
+     * Updates index. Note that key is unique for cache, so if cache contains multiple indexes
      * the key should be removed from indexes other than one being updated.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @param key Key.
      * @param val Value.
@@ -211,47 +211,36 @@ public interface GridQueryIndexing {
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteCheckedException If failed.
      */
-    public void store(String spaceName,
-        String typeName,
-        KeyCacheObject key,
-        int partId,
-        CacheObject val,
-        GridCacheVersion ver,
-        long expirationTime,
-        long link) throws IgniteCheckedException;
+    public void store(String cacheName, String typeName, KeyCacheObject key, int partId, CacheObject val,
+        GridCacheVersion ver, long expirationTime, long link) throws IgniteCheckedException;
 
     /**
      * Removes index entry by key.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(String spaceName,
-        GridQueryTypeDescriptor type,
-        KeyCacheObject key,
-        int partId,
-        CacheObject val,
+    public void remove(String cacheName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, CacheObject val,
         GridCacheVersion ver) throws IgniteCheckedException;
 
     /**
      * Rebuilds all indexes of given type from hash index.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param type Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    public void rebuildIndexesFromHash(String spaceName,
-        GridQueryTypeDescriptor type) throws IgniteCheckedException;
+    public void rebuildIndexesFromHash(String cacheName, GridQueryTypeDescriptor type) throws IgniteCheckedException;
 
     /**
      * Marks all indexes of given type for rebuild from hash index, making them unusable until rebuild finishes.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param type Type descriptor.
      */
-    public void markForRebuildFromHash(String spaceName, GridQueryTypeDescriptor type);
+    public void markForRebuildFromHash(String cacheName, GridQueryTypeDescriptor type);
 
     /**
      * Returns backup filter.
@@ -272,19 +261,19 @@ public interface GridQueryIndexing {
     /**
      * Prepare native statement to retrieve JDBC metadata from.
      *
-     * @param space Schema.
+     * @param cacheName Cache name.
      * @param sql Query.
      * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
      */
-    public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException;
+    public PreparedStatement prepareNativeStatement(String cacheName, String sql) throws SQLException;
 
     /**
-     * Gets space name from database schema.
+     * Gets cache name from database schema.
      *
      * @param schemaName Schema name. Could not be null. Could be empty.
-     * @return Space name. Could be null.
+     * @return Cache name. Could be null.
      */
-    public String space(String schemaName);
+    public String cacheName(String schemaName);
 
     /**
      * Collect queries that already running more than specified duration.
@@ -307,7 +296,7 @@ public interface GridQueryIndexing {
     public void cancelAllQueries();
 
     /**
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param nativeStmt Native statement.
      * @param autoFlushFreq Automatic data flushing frequency, disabled if {@code 0}.
      * @param nodeBufSize Per node buffer size - see {@link IgniteDataStreamer#perNodeBufferSize(int)}
@@ -316,6 +305,6 @@ public interface GridQueryIndexing {
      * @return {@link IgniteDataStreamer} tailored to specific needs of given native statement based on its metadata;
      * {@code null} if given statement is a query.
      */
-    public IgniteDataStreamer<?,?> createStreamer(String spaceName, PreparedStatement nativeStmt, long autoFlushFreq,
+    public IgniteDataStreamer<?,?> createStreamer(String cacheName, PreparedStatement nativeStmt, long autoFlushFreq,
         int nodeBufSize, int nodeParOps, boolean allowOverwrite);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 0df0f52..ced78cf 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
@@ -166,7 +166,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /** Coordinator node (initialized lazily). */
     private ClusterNode crd;
 
-    /** Registered spaces. */
+    /** Registered cache names. */
     private final Collection<String> cacheNames = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
 
     /** ID history for index create/drop discovery messages. */
@@ -668,7 +668,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         try {
             synchronized (stateMux) {
-                String space = cctx.name();
+                String cacheName = cctx.name();
 
                 // Prepare candidates.
                 List<Class<?>> mustDeserializeClss = new ArrayList<>();
@@ -679,7 +679,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 if (!F.isEmpty(qryEntities)) {
                     for (QueryEntity qryEntity : qryEntities) {
-                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(space, cctx, qryEntity,
+                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(cacheName, cctx, qryEntity,
                             mustDeserializeClss);
 
                         cands.add(cand);
@@ -696,14 +696,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     QueryTypeDescriptorImpl oldDesc = tblTypMap.put(desc.tableName(), desc);
 
                     if (oldDesc != null)
-                        throw new IgniteException("Duplicate table name [cache=" + space +
+                        throw new IgniteException("Duplicate table name [cache=" + cacheName +
                             ", tblName=" + desc.tableName() + ", type1=" + desc.name() + ", type2=" + oldDesc.name() + ']');
 
                     for (String idxName : desc.indexes().keySet()) {
                         oldDesc = idxTypMap.put(idxName, desc);
 
                         if (oldDesc != null)
-                            throw new IgniteException("Duplicate index name [cache=" + space +
+                            throw new IgniteException("Duplicate index name [cache=" + cacheName +
                                 ", idxName=" + idxName + ", type1=" + desc.name() + ", type2=" + oldDesc.name() + ']');
                     }
                 }
@@ -755,7 +755,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
 
                 // Ready to register at this point.
-                registerCache0(space, cctx, cands);
+                registerCache0(cacheName, cctx, cands);
 
                 // Warn about possible implicit deserialization.
                 if (!mustDeserializeClss.isEmpty()) {
@@ -1268,7 +1268,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * Register cache in indexing SPI.
      *
-     * @param cacheName Space.
+     * @param cacheName Cache name.
      * @param cctx Cache context.
      * @param cands Candidates.
      * @throws IgniteCheckedException If failed.
@@ -1324,7 +1324,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * Unregister cache.<p>
      * Use with {@link #busyLock} where appropriate.
      *
-     * @param cacheName Space.
+     * @param cacheName Cache name.
      */
     public void onCacheStop0(String cacheName) {
         if (idx == null)
@@ -1337,7 +1337,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             while (it.hasNext()) {
                 Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> entry = it.next();
 
-                if (F.eq(cacheName, entry.getKey().space())) {
+                if (F.eq(cacheName, entry.getKey().cacheName())) {
                     it.remove();
 
                     typesByName.remove(new QueryTypeNameKey(cacheName, entry.getValue().name()));
@@ -1354,7 +1354,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 QueryIndexKey idxKey = idxEntry.getKey();
 
-                if (F.eq(cacheName, idxKey.space()))
+                if (F.eq(cacheName, idxKey.cacheName()))
                     idxIt.remove();
             }
 
@@ -1377,21 +1377,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Check whether provided key and value belongs to expected space and table.
+     * Check whether provided key and value belongs to expected cache and table.
      *
      * @param cctx Target cache context.
-     * @param expSpace Expected space.
+     * @param expCacheName Expected cache name.
      * @param expTblName Expected table name.
      * @param key Key.
      * @param val Value.
-     * @return {@code True} if this key-value pair belongs to expected space/table, {@code false} otherwise or
-     *     if space or table doesn't exist.
+     * @return {@code True} if this key-value pair belongs to expected cache/table, {@code false} otherwise or
+     *     if cache or table doesn't exist.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ConstantConditions")
-    public boolean belongsToTable(GridCacheContext cctx, String expSpace, String expTblName, KeyCacheObject key,
+    public boolean belongsToTable(GridCacheContext cctx, String expCacheName, String expTblName, KeyCacheObject key,
         CacheObject val) throws IgniteCheckedException {
-        QueryTypeDescriptorImpl desc = type(expSpace, val);
+        QueryTypeDescriptorImpl desc = type(expCacheName, val);
 
         if (desc == null)
             return false;
@@ -1424,14 +1424,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     public IgniteInternalFuture<?> rebuildIndexesFromHash(Collection<Integer> cacheIds) {
         if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to get space size (grid is stopping).");
+            throw new IllegalStateException("Failed to rebuild indexes from hash (grid is stopping).");
 
         try {
             GridCompoundFuture<Object, ?> fut = new GridCompoundFuture<Object, Object>();
 
             for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet()) {
-                if (cacheIds.contains(CU.cacheId(e.getKey().space())))
-                    fut.add(rebuildIndexesFromHash(e.getKey().space(), e.getValue()));
+                if (cacheIds.contains(CU.cacheId(e.getKey().cacheName())))
+                    fut.add(rebuildIndexesFromHash(e.getKey().cacheName(), e.getValue()));
             }
 
             fut.markInitialized();
@@ -1444,12 +1444,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param desc Type descriptor.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
     private IgniteInternalFuture<Object> rebuildIndexesFromHash(
-        @Nullable final String space,
+        @Nullable final String cacheName,
         @Nullable final QueryTypeDescriptorImpl desc
     ) {
         if (idx == null)
@@ -1460,12 +1460,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         final GridWorkerFuture<Object> fut = new GridWorkerFuture<>();
 
-        idx.markForRebuildFromHash(space, desc);
+        idx.markForRebuildFromHash(cacheName, desc);
 
         GridWorker w = new GridWorker(ctx.igniteInstanceName(), "index-rebuild-worker", log) {
             @Override protected void body() {
                 try {
-                    idx.rebuildIndexesFromHash(space, desc);
+                    idx.rebuildIndexesFromHash(cacheName, desc);
 
                     fut.onDone();
                 }
@@ -1490,17 +1490,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @return Cache object context.
      */
-    private CacheObjectContext cacheObjectContext(String space) {
-        return ctx.cache().internalCache(space).context().cacheObjectContext();
+    private CacheObjectContext cacheObjectContext(String cacheName) {
+        return ctx.cache().internalCache(cacheName).context().cacheObjectContext();
     }
 
     /**
      * Writes key-value pair to index.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @param ver Cache entry version.
@@ -1508,7 +1508,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    public void store(final String space,
+    public void store(final String cacheName,
         final KeyCacheObject key,
         int partId,
         @Nullable CacheObject prevVal,
@@ -1521,7 +1521,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         assert val != null;
 
         if (log.isDebugEnabled())
-            log.debug("Store [space=" + space + ", key=" + key + ", val=" + val + "]");
+            log.debug("Store [cache=" + cacheName + ", key=" + key + ", val=" + val + "]");
 
         if (idx == null)
             return;
@@ -1530,7 +1530,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 
         try {
-            CacheObjectContext coctx = cacheObjectContext(space);
+            CacheObjectContext coctx = cacheObjectContext(cacheName);
 
             QueryTypeDescriptorImpl desc = typeByValue(coctx, key, val, true);
 
@@ -1538,13 +1538,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 QueryTypeDescriptorImpl prevValDesc = typeByValue(coctx, key, prevVal, false);
 
                 if (prevValDesc != null && prevValDesc != desc)
-                    idx.remove(space, prevValDesc, key, partId, prevVal, prevVer);
+                    idx.remove(cacheName, prevValDesc, key, partId, prevVal, prevVer);
             }
 
             if (desc == null)
                 return;
 
-            idx.store(space, desc.name(), key, partId, val, ver, expirationTime, link);
+            idx.store(cacheName, desc.name(), key, partId, val, ver, expirationTime, link);
         }
         finally {
             busyLock.leaveBusy();
@@ -1606,25 +1606,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets type descriptor for space by given object's type.
+     * Gets type descriptor for cache by given object's type.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param val Object to determine type for.
      * @return Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ConstantConditions")
-    private QueryTypeDescriptorImpl type(@Nullable String space, CacheObject val) throws IgniteCheckedException {
-        CacheObjectContext coctx = cacheObjectContext(space);
+    private QueryTypeDescriptorImpl type(@Nullable String cacheName, CacheObject val) throws IgniteCheckedException {
+        CacheObjectContext coctx = cacheObjectContext(cacheName);
 
         QueryTypeIdKey id;
 
         boolean binaryVal = ctx.cacheObjects().isBinaryObject(val);
 
         if (binaryVal)
-            id = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(val));
+            id = new QueryTypeIdKey(cacheName, ctx.cacheObjects().typeId(val));
         else
-            id = new QueryTypeIdKey(space, val.value(coctx, false).getClass());
+            id = new QueryTypeIdKey(cacheName, val.value(coctx, false).getClass());
 
         return types.get(id);
     }
@@ -1718,12 +1718,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param spaceName Cache name.
+     * @param cacheName Cache name.
      * @param streamer Data streamer.
      * @param qry Query.
      * @return Iterator.
      */
-    public long streamUpdateQuery(@Nullable final String spaceName,
+    public long streamUpdateQuery(@Nullable final String cacheName,
         final IgniteDataStreamer<?, ?> streamer, final String qry, final Object[] args) {
         assert streamer != null;
 
@@ -1731,11 +1731,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            GridCacheContext cctx = ctx.cache().cache(spaceName).context();
+            GridCacheContext cctx = ctx.cache().cache(cacheName).context();
 
             return executeQuery(GridCacheQueryType.SQL_FIELDS, qry, cctx, new IgniteOutClosureX<Long>() {
                 @Override public Long applyx() throws IgniteCheckedException {
-                    return idx.streamUpdateQuery(spaceName, qry, args, streamer);
+                    return idx.streamUpdateQuery(cacheName, qry, args, streamer);
                 }
             }, true);
         }
@@ -1966,28 +1966,28 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
     /**
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param sql Query.
      * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
      */
-    public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException {
+    public PreparedStatement prepareNativeStatement(String cacheName, String sql) throws SQLException {
         checkxEnabled();
 
-        return idx.prepareNativeStatement(space, sql);
+        return idx.prepareNativeStatement(cacheName, sql);
     }
 
     /**
      * @param schema Schema name.
-     * @return space (cache) name from schema name.
+     * @return Cache name from schema name.
      */
-    public String space(String schema) throws SQLException {
+    public String cacheName(String schema) throws SQLException {
         checkxEnabled();
 
-        return idx.space(schema);
+        return idx.cacheName(schema);
     }
 
     /**
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param nativeStmt Native statement.
      * @param autoFlushFreq Automatic data flushing frequency, disabled if {@code 0}.
      * @param nodeBufSize Per node buffer size - see {@link IgniteDataStreamer#perNodeBufferSize(int)}
@@ -1996,21 +1996,22 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @see IgniteDataStreamer#allowOverwrite
      * @return {@link IgniteDataStreamer} tailored to specific needs of given native statement based on its metadata.
      */
-    public IgniteDataStreamer<?, ?> createStreamer(String spaceName, PreparedStatement nativeStmt, long autoFlushFreq,
+    public IgniteDataStreamer<?, ?> createStreamer(String cacheName, PreparedStatement nativeStmt, long autoFlushFreq,
         int nodeBufSize, int nodeParOps, boolean allowOverwrite) {
-        return idx.createStreamer(spaceName, nativeStmt, autoFlushFreq, nodeBufSize, nodeParOps, allowOverwrite);
+        return idx.createStreamer(cacheName, nativeStmt, autoFlushFreq, nodeBufSize, nodeParOps, allowOverwrite);
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void remove(String space, KeyCacheObject key, int partId, CacheObject val, GridCacheVersion ver) throws IgniteCheckedException {
+    public void remove(String cacheName, KeyCacheObject key, int partId, CacheObject val, GridCacheVersion ver)
+        throws IgniteCheckedException {
         assert key != null;
 
         if (log.isDebugEnabled())
-            log.debug("Remove [space=" + space + ", key=" + key + ", val=" + val + "]");
+            log.debug("Remove [cacheName=" + cacheName + ", key=" + key + ", val=" + val + "]");
 
         if (idx == null)
             return;
@@ -2019,14 +2020,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to remove from index (grid is stopping).");
 
         try {
-            CacheObjectContext coctx = cacheObjectContext(space);
+            CacheObjectContext coctx = cacheObjectContext(cacheName);
 
             QueryTypeDescriptorImpl desc = typeByValue(coctx, key, val, false);
 
             if (desc == null)
                 return;
 
-            idx.remove(space, desc, key, partId, val, ver);
+            idx.remove(cacheName, desc, key, partId, val, ver);
         }
         finally {
             busyLock.leaveBusy();
@@ -2034,7 +2035,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param clause Clause.
      * @param resType Result type.
      * @param filters Key and value filters.
@@ -2044,7 +2045,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String space, final String clause,
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String cacheName, final String clause,
         final String resType, final IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
@@ -2052,14 +2053,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(cacheName).context();
 
             return executeQuery(GridCacheQueryType.TEXT, clause, cctx,
                 new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                     @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                        String typeName = typeName(space, resType);
+                        String typeName = typeName(cacheName, resType);
 
-                        return idx.queryLocalText(space, clause, typeName, filters);
+                        return idx.queryLocalText(cacheName, clause, typeName, filters);
                     }
                 }, true);
         }
@@ -2069,33 +2070,33 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets types for space.
+     * Gets types for cache.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @return Descriptors.
      */
-    public Collection<GridQueryTypeDescriptor> types(@Nullable String space) {
-        Collection<GridQueryTypeDescriptor> spaceTypes = new ArrayList<>();
+    public Collection<GridQueryTypeDescriptor> types(@Nullable String cacheName) {
+        Collection<GridQueryTypeDescriptor> cacheTypes = new ArrayList<>();
 
         for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet()) {
             QueryTypeDescriptorImpl desc = e.getValue();
 
-            if (F.eq(e.getKey().space(), space))
-                spaceTypes.add(desc);
+            if (F.eq(e.getKey().cacheName(), cacheName))
+                cacheTypes.add(desc);
         }
 
-        return spaceTypes;
+        return cacheTypes;
     }
 
     /**
-     * Get type descriptor for the given space and table name.
-     * @param space Space.
+     * Get type descriptor for the given cache and table name.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @return Type (if any).
      */
-    @Nullable private QueryTypeDescriptorImpl type(@Nullable String space, String tblName) {
+    @Nullable private QueryTypeDescriptorImpl type(@Nullable String cacheName, String tblName) {
         for (QueryTypeDescriptorImpl type : types.values()) {
-            if (F.eq(space, type.space()) && F.eq(tblName, type.tableName()))
+            if (F.eq(cacheName, type.cacheName()) && F.eq(tblName, type.tableName()))
                 return type;
         }
 
@@ -2103,15 +2104,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets type name for provided space and type name if type is still valid.
+     * Gets type name for provided cache name and type name if type is still valid.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @return Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private String typeName(@Nullable String space, String typeName) throws IgniteCheckedException {
-        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, typeName));
+    private String typeName(@Nullable String cacheName, String typeName) throws IgniteCheckedException {
+        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(cacheName, typeName));
 
         if (type == null)
             throw new IgniteCheckedException("Failed to find SQL table for type: " + typeName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
index f580111..7fdb805 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
@@ -29,8 +29,8 @@ public class QueryIndexKey implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Space. */
-    private final String space;
+    /** Cache name. */
+    private final String cacheName;
 
     /** Name. */
     private final String name;
@@ -38,19 +38,19 @@ public class QueryIndexKey implements Serializable {
     /**
      * Constructor.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param name Name.
      */
-    public QueryIndexKey(String space, String name) {
-        this.space = space;
+    public QueryIndexKey(String cacheName, String name) {
+        this.cacheName = cacheName;
         this.name = name;
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /**
@@ -62,7 +62,7 @@ public class QueryIndexKey implements Serializable {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (space != null ? space.hashCode() : 0) + (name != null ? name.hashCode() : 0);
+        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + (name != null ? name.hashCode() : 0);
     }
 
     /** {@inheritDoc} */
@@ -75,7 +75,7 @@ public class QueryIndexKey implements Serializable {
 
         QueryIndexKey other = (QueryIndexKey)o;
 
-        return F.eq(name, other.name) && F.eq(space, other.space);
+        return F.eq(name, other.name) && F.eq(cacheName, other.cacheName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index 56c6aa5..4848b0a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -36,8 +36,8 @@ import java.util.Map;
  * Descriptor of type.
  */
 public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
-    /** Space. */
-    private final String space;
+    /** Cache name. */
+    private final String cacheName;
 
     /** */
     private String name;
@@ -102,17 +102,17 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     /**
      * Constructor.
      *
-     * @param space Cache name.
+     * @param cacheName Cache name.
      */
-    public QueryTypeDescriptorImpl(String space) {
-        this.space = space;
+    public QueryTypeDescriptorImpl(String cacheName) {
+        this.cacheName = cacheName;
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
index 4d486f9..fe7c487 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.query;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * Identifying TypeDescriptor by space and value class.
+ * Identifying TypeDescriptor by cache name and value class.
  */
 public class QueryTypeIdKey {
     /** */
-    private final String space;
+    private final String cacheName;
 
     /** Value type. */
     private final Class<?> valType;
@@ -35,13 +35,13 @@ public class QueryTypeIdKey {
     /**
      * Constructor.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param valType Value type.
      */
-    public  QueryTypeIdKey(String space, Class<?> valType) {
+    public  QueryTypeIdKey(String cacheName, Class<?> valType) {
         assert valType != null;
 
-        this.space = space;
+        this.cacheName = cacheName;
         this.valType = valType;
 
         valTypeId = 0;
@@ -50,21 +50,21 @@ public class QueryTypeIdKey {
     /**
      * Constructor.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param valTypeId Value type ID.
      */
-    public QueryTypeIdKey(String space, int valTypeId) {
-        this.space = space;
+    public QueryTypeIdKey(String cacheName, int valTypeId) {
+        this.cacheName = cacheName;
         this.valTypeId = valTypeId;
 
         valType = null;
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /** {@inheritDoc} */
@@ -79,12 +79,12 @@ public class QueryTypeIdKey {
 
         return (valTypeId == typeId.valTypeId) &&
             (valType != null ? valType == typeId.valType : typeId.valType == null) &&
-            (space != null ? space.equals(typeId.space) : typeId.space == null);
+            (cacheName != null ? cacheName.equals(typeId.cacheName) : typeId.cacheName == null);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (space != null ? space.hashCode() : 0) + (valType != null ? valType.hashCode() : valTypeId);
+        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + (valType != null ? valType.hashCode() : valTypeId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
index 8a36a03..b10c5b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
@@ -26,19 +26,19 @@ import org.jetbrains.annotations.Nullable;
  */
 public class QueryTypeNameKey {
     /** */
-    private final String space;
+    private final String cacheName;
 
     /** */
     private final String typeName;
 
     /**
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      */
-    public QueryTypeNameKey(@Nullable String space, String typeName) {
+    public QueryTypeNameKey(@Nullable String cacheName, String typeName) {
         assert !F.isEmpty(typeName) : typeName;
 
-        this.space = space;
+        this.cacheName = cacheName;
         this.typeName = typeName;
     }
 
@@ -52,13 +52,13 @@ public class QueryTypeNameKey {
 
         QueryTypeNameKey other = (QueryTypeNameKey)o;
 
-        return (space != null ? space.equals(other.space) : other.space == null) &&
+        return (cacheName != null ? cacheName.equals(other.cacheName) : other.cacheName == null) &&
             typeName.equals(other.typeName);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (space != null ? space.hashCode() : 0) + typeName.hashCode();
+        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + typeName.hashCode();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 1a80a37..245965c 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
@@ -163,14 +163,14 @@ public class QueryUtils {
     /**
      * Create type candidate for query entity.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param cctx Cache context.
      * @param qryEntity Query entity.
      * @param mustDeserializeClss Classes which must be deserialized.
      * @return Type candidate.
      * @throws IgniteCheckedException If failed.
      */
-    public static QueryTypeCandidate typeForQueryEntity(String space, GridCacheContext cctx, QueryEntity qryEntity,
+    public static QueryTypeCandidate typeForQueryEntity(String cacheName, GridCacheContext cctx, QueryEntity qryEntity,
         List<Class<?>> mustDeserializeClss) throws IgniteCheckedException {
         GridKernalContext ctx = cctx.kernalContext();
         CacheConfiguration<?,?> ccfg = cctx.config();
@@ -179,7 +179,7 @@ public class QueryUtils {
 
         CacheObjectContext coCtx = binaryEnabled ? ctx.cacheObjects().contextForCache(ccfg) : null;
 
-        QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl(space);
+        QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl(cacheName);
 
         desc.aliases(qryEntity.getAliases());
 
@@ -244,10 +244,10 @@ public class QueryUtils {
         if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
             processBinaryMeta(ctx, qryEntity, desc);
 
-            typeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(qryEntity.findValueType()));
+            typeId = new QueryTypeIdKey(cacheName, ctx.cacheObjects().typeId(qryEntity.findValueType()));
 
             if (valCls != null)
-                altTypeId = new QueryTypeIdKey(space, valCls);
+                altTypeId = new QueryTypeIdKey(cacheName, valCls);
 
             if (!cctx.customAffinityMapper() && qryEntity.findKeyType() != null) {
                 // Need to setup affinity key for distributed joins.
@@ -270,8 +270,8 @@ public class QueryUtils {
                     desc.affinityKey(affField);
             }
 
-            typeId = new QueryTypeIdKey(space, valCls);
-            altTypeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(qryEntity.findValueType()));
+            typeId = new QueryTypeIdKey(cacheName, valCls);
+            altTypeId = new QueryTypeIdKey(cacheName, ctx.cacheObjects().typeId(qryEntity.findValueType()));
         }
 
         return new QueryTypeCandidate(typeId, altTypeId, desc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
index 58c909d..b3fa47c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
@@ -48,8 +48,8 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
     /** Cache context. */
     private final GridCacheContext cctx;
 
-    /** Space name. */
-    private final String spaceName;
+    /** Cache name. */
+    private final String cacheName;
 
     /** Table name. */
     private final String tblName;
@@ -61,14 +61,14 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
      * Constructor.
      *
      * @param cctx Cache context.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @param cancel Cancellation token.
      */
-    public SchemaIndexCacheVisitorImpl(GridQueryProcessor qryProc, GridCacheContext cctx, String spaceName,
+    public SchemaIndexCacheVisitorImpl(GridQueryProcessor qryProc, GridCacheContext cctx, String cacheName,
         String tblName, SchemaIndexOperationCancellationToken cancel) {
         this.qryProc = qryProc;
-        this.spaceName = spaceName;
+        this.cacheName = cacheName;
         this.tblName = tblName;
         this.cancel = cancel;
 
@@ -190,7 +190,7 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         /** {@inheritDoc} */
         @Override public void apply(KeyCacheObject key, int part, CacheObject val, GridCacheVersion ver,
             long expiration, long link) throws IgniteCheckedException {
-            if (qryProc.belongsToTable(cctx, spaceName, tblName, key, val))
+            if (qryProc.belongsToTable(cctx, cacheName, tblName, key, val))
                 target.apply(key, part, val, ver, expiration, link);
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
index 726c90c..c45e229 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
@@ -58,7 +58,7 @@ public abstract class SchemaAbstractOperation implements Serializable {
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
     public String cacheName() {
         return cacheName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
index 935feab..74d349a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
@@ -25,17 +25,17 @@ import org.jetbrains.annotations.Nullable;
  */
 public interface IndexingQueryFilter {
     /**
-     * Creates optional predicate for space.
+     * Creates optional predicate for cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @return Predicate or {@code null} if no filtering is needed.
      */
-    @Nullable public <K, V> IgniteBiPredicate<K, V> forSpace(String spaceName);
+    @Nullable public <K, V> IgniteBiPredicate<K, V> forCache(String cacheName);
 
     /**
      * Is the value required for filtering logic?
      * If false then null instead of value will be passed
-     * to IgniteBiPredicate returned by {@link #forSpace(String)} method.
+     * to IgniteBiPredicate returned by {@link #forCache(String)} method.
      *
      * @return true if value is required for filtering, false otherwise.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
index 4d53bea..8ec4a67 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
@@ -25,8 +25,7 @@ import org.apache.ignite.spi.IgniteSpiException;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Indexing SPI allows user to index cache content. Using indexing SPI user can index data in cache and run
- * Usually cache name will be used as space name, so multiple caches can write to single indexing SPI instance.
+ * Indexing SPI allows user to index cache content. Using indexing SPI user can index data in cache and run queries.
  * <p>
  * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
  * internal view on the subsystem and is used internally by Ignite kernal. In rare use cases when
@@ -66,33 +65,33 @@ public interface IndexingSpi extends IgniteSpi {
     /**
      * Executes query.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param params Query parameters.
      * @param filters System filters.
      * @return Query result. If the iterator implements {@link AutoCloseable} it will be correctly closed.
      * @throws IgniteSpiException If failed.
      */
-    public Iterator<Cache.Entry<?,?>> query(@Nullable String spaceName, Collection<Object> params,
+    public Iterator<Cache.Entry<?,?>> query(@Nullable String cacheName, Collection<Object> params,
         @Nullable IndexingQueryFilter filters) throws IgniteSpiException;
 
     /**
-     * Updates index. Note that key is unique for space, so if space contains multiple indexes
+     * Updates index. Note that key is unique for cache, so if cache contains multiple indexes
      * the key should be removed from indexes other than one being updated.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteSpiException If failed.
      */
-    public void store(@Nullable String spaceName, Object key, Object val, long expirationTime) throws IgniteSpiException;
+    public void store(@Nullable String cacheName, Object key, Object val, long expirationTime) throws IgniteSpiException;
 
     /**
      * Removes index entry by key.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param key Key.
      * @throws IgniteSpiException If failed.
      */
-    public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException;
+    public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
index 0ed7e33..5677f55 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
@@ -33,19 +33,19 @@ import org.jetbrains.annotations.Nullable;
 @IgniteSpiNoop
 public class NoopIndexingSpi extends IgniteSpiAdapter implements IndexingSpi {
     /** {@inheritDoc} */
-    @Override public Iterator<Cache.Entry<?,?>> query(@Nullable String spaceName, Collection<Object> params,
+    @Override public Iterator<Cache.Entry<?,?>> query(@Nullable String cacheName, Collection<Object> params,
         @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
         throw new IgniteSpiException("You have to configure custom GridIndexingSpi implementation.");
     }
 
     /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+    @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
         throws IgniteSpiException {
         assert false;
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+    @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
         assert false;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
index 1d27524..ac294b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
@@ -664,13 +664,13 @@ public abstract class IgniteTxExceptionAbstractSelfTest extends GridCacheAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
             throw new UnsupportedOperationException();
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
             throws IgniteSpiException {
             if (fail) {
                 fail = false;
@@ -680,7 +680,7 @@ public abstract class IgniteTxExceptionAbstractSelfTest extends GridCacheAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object k)
+        @Override public void remove(@Nullable String cacheName, Object k)
             throws IgniteSpiException {
             if (fail) {
                 fail = false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
index 7349a4e..b6e32d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
@@ -254,7 +254,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
             if (params.size() < 2)
                 throw new IgniteSpiException("Range parameters required.");
@@ -278,7 +278,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
             throws IgniteSpiException {
             assertFalse(key instanceof BinaryObject);
             assertFalse(val instanceof BinaryObject);
@@ -287,7 +287,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
             // No-op.
         }
     }
@@ -298,17 +298,17 @@ public class IndexingSpiQuerySelfTest extends TestCase {
     private static class MyBinaryIndexingSpi extends MyIndexingSpi {
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val,
+        @Override public void store(@Nullable String cacheName, Object key, Object val,
             long expirationTime) throws IgniteSpiException {
             assertTrue(key instanceof BinaryObject);
 
             assertTrue(val instanceof BinaryObject);
 
-            super.store(spaceName, ((BinaryObject)key).deserialize(), ((BinaryObject)val).deserialize(), expirationTime);
+            super.store(cacheName, ((BinaryObject)key).deserialize(), ((BinaryObject)val).deserialize(), expirationTime);
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
             assertTrue(key instanceof BinaryObject);
         }
     }
@@ -318,7 +318,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
      */
     private static class MyBrokenIndexingSpi extends MyIndexingSpi {
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val,
+        @Override public void store(@Nullable String cacheName, Object key, Object val,
             long expirationTime) throws IgniteSpiException {
             throw new IgniteSpiException("Test exception");
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
index 9d2b31c..e59deed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
@@ -133,19 +133,19 @@ public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
            return null;
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
             throws IgniteSpiException {
             throw new IgniteSpiException("Test exception");
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 2a3d77c..db7bfd6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -122,31 +122,31 @@ public class DmlStatementsProcessor {
     /**
      * Handle cache stop.
      *
-     * @param spaceName Cache name.
+     * @param cacheName Cache name.
      */
-    public void onCacheStop(String spaceName) {
-        planCache.remove(spaceName);
+    public void onCacheStop(String cacheName) {
+        planCache.remove(cacheName);
     }
 
     /**
      * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param stmt JDBC statement.
      * @param fieldsQry Original query.
      * @param loc Query locality flag.
-     * @param filters Space name and key filter.
+     * @param filters Cache name and key filter.
      * @param cancel Cancel.
      * @return Update result (modified items count and failed keys).
      * @throws IgniteCheckedException if failed.
      */
-    private UpdateResult updateSqlFields(String spaceName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
+    private UpdateResult updateSqlFields(String cacheName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
         boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
         Object[] errKeys = null;
 
         long items = 0;
 
-        UpdatePlan plan = getPlanForStatement(spaceName, stmt, null);
+        UpdatePlan plan = getPlanForStatement(cacheName, stmt, null);
 
         GridCacheContext<?, ?> cctx = plan.tbl.rowDescriptor().context();
 
@@ -194,7 +194,7 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param stmt Prepared statement.
      * @param fieldsQry Initial query.
      * @param cancel Query cancel.
@@ -202,9 +202,9 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String spaceName, PreparedStatement stmt,
+    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String cacheName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(spaceName, stmt, fieldsQry, false, null, cancel);
+        UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, false, null, cancel);
 
         QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
             (Collections.singletonList(res.cnt)), null, false);
@@ -216,17 +216,17 @@ public class DmlStatementsProcessor {
 
     /**
      * Execute DML statement on local cache.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param stmt Prepared statement.
-     * @param filters Space name and key filter.
+     * @param filters Cache name and key filter.
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    GridQueryFieldsResult updateLocalSqlFields(String spaceName, PreparedStatement stmt,
+    GridQueryFieldsResult updateLocalSqlFields(String cacheName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(spaceName, stmt, fieldsQry, true, filters, cancel);
+        UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, true, filters, cancel);
 
         return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
             new IgniteSingletonIterator(Collections.singletonList(res.cnt)));
@@ -317,8 +317,9 @@ public class DmlStatementsProcessor {
      * Actually perform SQL DML operation locally.
      * @param cctx Cache context.
      * @param prepStmt Prepared statement for DML query.
-     * @param filters Space name and key filter.
-     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.   @return Pair [number of successfully processed items; keys that have failed to be processed]
+     * @param filters Cache name and key filter.
+     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
+     * @return Pair [number of successfully processed items; keys that have failed to be processed]
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
@@ -392,28 +393,28 @@ public class DmlStatementsProcessor {
     /**
      * Generate SELECT statements to retrieve data for modifications from and find fast UPDATE or DELETE args,
      * if available.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param prepStmt JDBC statement.
      * @return Update plan.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    private UpdatePlan getPlanForStatement(String spaceName, PreparedStatement prepStmt,
+    private UpdatePlan getPlanForStatement(String cacheName, PreparedStatement prepStmt,
         @Nullable Integer errKeysPos) throws IgniteCheckedException {
         Prepared p = GridSqlQueryParser.prepared(prepStmt);
 
-        spaceName = F.isEmpty(spaceName) ? "default" : spaceName;
+        cacheName = F.isEmpty(cacheName) ? "default" : cacheName;
 
-        ConcurrentMap<String, UpdatePlan> spacePlans = planCache.get(spaceName);
+        ConcurrentMap<String, UpdatePlan> cachePlans = planCache.get(cacheName);
 
-        if (spacePlans == null) {
-            spacePlans = new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
+        if (cachePlans == null) {
+            cachePlans = new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
 
-            spacePlans = U.firstNotNull(planCache.putIfAbsent(spaceName, spacePlans), spacePlans);
+            cachePlans = U.firstNotNull(planCache.putIfAbsent(cacheName, cachePlans), cachePlans);
         }
 
         // getSQL returns field value, so it's fast
         // Don't look for re-runs in cache, we don't cache them
-        UpdatePlan res = (errKeysPos == null ? spacePlans.get(p.getSQL()) : null);
+        UpdatePlan res = (errKeysPos == null ? cachePlans.get(p.getSQL()) : null);
 
         if (res != null)
             return res;
@@ -422,7 +423,7 @@ public class DmlStatementsProcessor {
 
         // Don't cache re-runs
         if (errKeysPos == null)
-            return U.firstNotNull(spacePlans.putIfAbsent(p.getSQL(), res), res);
+            return U.firstNotNull(cachePlans.putIfAbsent(p.getSQL(), res), res);
         else
             return res;
     }


[13/39] ignite git commit: Typo

Posted by sb...@apache.org.
Typo


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 8c9b1bde2884e05e4f0cc4349606250a5bc37aa1
Parents: b131ff0
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed May 24 11:01:26 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed May 24 11:01:26 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/communication/GridIoManager.java      | 2 +-
 .../internal/managers/communication/IgniteIoTestMessage.java       | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8c9b1bde/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 68bfd07..698baf8 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
@@ -3067,7 +3067,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 maxReqWireTimeTs = now;
             }
 
-            long resWireTimeMillis = msg.responseRecievedTsMillis() - msg.requestSendTsMillis();
+            long resWireTimeMillis = msg.responseReceivedTsMillis() - msg.requestSendTsMillis();
 
             if (maxResWireTimeMillis < resWireTimeMillis) {
                 maxResWireTimeMillis = resWireTimeMillis;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c9b1bde/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
index 3e0fa76..0a8b2b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
@@ -240,7 +240,7 @@ public class IgniteIoTestMessage implements Message {
     /**
      * @return Response received timestamp (millis).
      */
-    public long responseRecievedTsMillis() {
+    public long responseReceivedTsMillis() {
         return resRcvTsMillis;
     }
 


[02/39] ignite git commit: IGNITE-5163: Implemented infrastructure for the new JDBC driver. This closes #1912.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectWriter.java
new file mode 100644
index 0000000..f5e9924
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectWriter.java
@@ -0,0 +1,111 @@
+/*
+ * 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.odbc;
+
+import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.UUID;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Binary writer with marshaling non-primitive and non-embedded objects with JDK marshaller..
+ */
+public abstract class SqlListenerAbstractObjectWriter {
+    /**
+     * @param writer Writer.
+     * @param obj Object to write.
+     * @throws BinaryObjectException On error.
+     */
+    public void writeObject(BinaryWriterExImpl writer, @Nullable Object obj) throws BinaryObjectException {
+        if (obj == null) {
+            writer.writeByte(GridBinaryMarshaller.NULL);
+
+            return;
+        }
+
+        Class<?> cls = obj.getClass();
+
+        if (cls == Boolean.class)
+            writer.writeBooleanFieldPrimitive((Boolean)obj);
+        else if (cls == Byte.class)
+            writer.writeByteFieldPrimitive((Byte)obj);
+        else if (cls == Character.class)
+            writer.writeCharFieldPrimitive((Character)obj);
+        else if (cls == Short.class)
+            writer.writeShortFieldPrimitive((Short)obj);
+        else if (cls == Integer.class)
+            writer.writeIntFieldPrimitive((Integer)obj);
+        else if (cls == Long.class)
+            writer.writeLongFieldPrimitive((Long)obj);
+        else if (cls == Float.class)
+            writer.writeFloatFieldPrimitive((Float)obj);
+        else if (cls == Double.class)
+            writer.writeDoubleFieldPrimitive((Double)obj);
+        else if (cls == String.class)
+            writer.doWriteString((String)obj);
+        else if (cls == BigDecimal.class)
+            writer.doWriteDecimal((BigDecimal)obj);
+        else if (cls == UUID.class)
+            writer.writeUuid((UUID)obj);
+        else if (cls == Time.class)
+            writer.writeTime((Time)obj);
+        else if (cls == Timestamp.class)
+            writer.writeTimestamp((Timestamp)obj);
+        else if (cls == java.sql.Date.class || cls == java.util.Date.class)
+            writer.writeDate((java.util.Date)obj);
+        else if (cls == boolean[].class)
+            writer.writeBooleanArray((boolean[])obj);
+        else if (cls == byte[].class)
+            writer.writeByteArray((byte[])obj);
+        else if (cls == char[].class)
+            writer.writeCharArray((char[])obj);
+        else if (cls == short[].class)
+            writer.writeShortArray((short[])obj);
+        else if (cls == int[].class)
+            writer.writeIntArray((int[])obj);
+        else if (cls == float[].class)
+            writer.writeFloatArray((float[])obj);
+        else if (cls == double[].class)
+            writer.writeDoubleArray((double[])obj);
+        else if (cls == String[].class)
+            writer.writeStringArray((String[])obj);
+        else if (cls == BigDecimal[].class)
+            writer.writeDecimalArray((BigDecimal[])obj);
+        else if (cls == UUID[].class)
+            writer.writeUuidArray((UUID[])obj);
+        else if (cls == Time[].class)
+            writer.writeTimeArray((Time[])obj);
+        else if (cls == Timestamp[].class)
+            writer.writeTimestampArray((Timestamp[])obj);
+        else if (cls == java.util.Date[].class || cls == java.sql.Date[].class)
+            writer.writeDateArray((java.util.Date[])obj);
+        else
+            writeCustomObject(writer, obj);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param obj Object to marshal with marshaller and write to binary stream.
+     * @throws BinaryObjectException On error.
+     */
+    protected abstract void writeCustomObject(BinaryWriterExImpl writer, Object obj) throws BinaryObjectException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
new file mode 100644
index 0000000..9eaec04
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
@@ -0,0 +1,263 @@
+/*
+ * 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.odbc;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
+import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMessageParser;
+import org.apache.ignite.internal.processors.odbc.odbc.OdbcMessageParser;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
+import org.apache.ignite.internal.util.nio.GridNioSession;
+import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * ODBC message listener.
+ */
+public class SqlListenerNioListener extends GridNioServerListenerAdapter<byte[]> {
+    /** The value corresponds to ODBC driver of the parser field of the handshake request. */
+    public static final byte ODBC_CLIENT = 0;
+
+    /** The value corresponds to JDBC driver of the parser field of the handshake request. */
+    public static final byte JDBC_CLIENT = 1;
+
+    /** Current version. */
+    private static final SqlListenerProtocolVersion CURRENT_VER = SqlListenerProtocolVersion.create(2, 1, 0);
+
+    /** Supported versions. */
+    private static final Set<SqlListenerProtocolVersion> SUPPORTED_VERS = new HashSet<>();
+
+    /** Connection-related metadata key. */
+    private static final int CONN_CTX_META_KEY = GridNioSessionMetaKey.nextUniqueKey();
+
+    /** Request ID generator. */
+    private static final AtomicLong REQ_ID_GEN = new AtomicLong();
+
+    /** Busy lock. */
+    private final GridSpinBusyLock busyLock;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Maximum allowed cursors. */
+    private final int maxCursors;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    static {
+        SUPPORTED_VERS.add(CURRENT_VER);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     * @param busyLock Shutdown busy lock.
+     * @param maxCursors Maximum allowed cursors.
+     */
+    public SqlListenerNioListener(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) {
+        this.ctx = ctx;
+        this.busyLock = busyLock;
+        this.maxCursors = maxCursors;
+
+        log = ctx.log(getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onConnected(GridNioSession ses) {
+        if (log.isDebugEnabled())
+            log.debug("SQL client connected: " + ses.remoteAddress());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
+        if (log.isDebugEnabled()) {
+            if (e == null)
+                log.debug("SQL client disconnected: " + ses.remoteAddress());
+            else
+                log.debug("SQL client disconnected due to an error [addr=" + ses.remoteAddress() + ", err=" + e + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMessage(GridNioSession ses, byte[] msg) {
+        assert msg != null;
+
+        SqlListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY);
+
+        if (connCtx == null) {
+            onHandshake(ses, msg);
+
+            return;
+        }
+
+        SqlListenerMessageParser parser = connCtx.parser();
+
+        SqlListenerRequest req;
+
+        try {
+            req = parser.decode(msg);
+        }
+        catch (Exception e) {
+            log.error("Failed to parse SQL client request [err=" + e + ']');
+
+            ses.close();
+
+            return;
+        }
+
+        assert req != null;
+
+        req.requestId(REQ_ID_GEN.incrementAndGet());
+
+        try {
+            long startTime = 0;
+
+            if (log.isDebugEnabled()) {
+                startTime = System.nanoTime();
+
+                log.debug("SQL client request received [reqId=" + req.requestId() + ", addr=" + ses.remoteAddress() +
+                    ", req=" + req + ']');
+            }
+
+            SqlListenerRequestHandler handler = connCtx.handler();
+
+            SqlListenerResponse resp = handler.handle(req);
+
+            if (log.isDebugEnabled()) {
+                long dur = (System.nanoTime() - startTime) / 1000;
+
+                log.debug("SQL client request processed [reqId=" + req.requestId() + ", dur(mcs)=" + dur  +
+                    ", resp=" + resp.status() + ']');
+            }
+
+            byte[] outMsg = parser.encode(resp);
+
+            ses.send(outMsg);
+        }
+        catch (Exception e) {
+            log.error("Failed to process SQL client request [reqId=" + req.requestId() + ", err=" + e + ']');
+
+            ses.send(parser.encode(new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString())));
+        }
+    }
+
+    /**
+     * Perform handshake.
+     *
+     * @param ses Session.
+     * @param msg Message bytes.
+     */
+    private void onHandshake(GridNioSession ses, byte[] msg) {
+        BinaryInputStream stream = new BinaryHeapInputStream(msg);
+
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null, true);
+
+        byte cmd = reader.readByte();
+
+        if (cmd != SqlListenerRequest.HANDSHAKE) {
+            log.error("Unexpected SQL client request (will close session): " + ses.remoteAddress());
+
+            ses.close();
+
+            return;
+        }
+
+        short verMajor = reader.readShort();
+        short verMinor = reader.readShort();
+        short verMaintenance = reader.readShort();
+
+        SqlListenerProtocolVersion ver = SqlListenerProtocolVersion.create(verMajor, verMinor, verMaintenance);
+
+        String errMsg = null;
+
+        if (SUPPORTED_VERS.contains(ver)) {
+            // Prepare context.
+            SqlListenerConnectionContext connCtx = prepareContext(ver, reader);
+
+            ses.addMeta(CONN_CTX_META_KEY, connCtx);
+        }
+        else {
+            log.warning("Unsupported version: " + ver.toString());
+
+            errMsg = "Unsupported version.";
+        }
+
+        // Send response.
+        BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(8), null, null);
+
+        if (errMsg == null)
+            writer.writeBoolean(true);
+        else {
+            writer.writeBoolean(false);
+            writer.writeShort(CURRENT_VER.major());
+            writer.writeShort(CURRENT_VER.minor());
+            writer.writeShort(CURRENT_VER.maintenance());
+            writer.doWriteString(errMsg);
+        }
+
+        ses.send(writer.array());
+    }
+
+    /**
+     * Prepare context.
+     *
+     * @param ver Version.
+     * @param reader Reader.
+     * @return Context.
+     */
+    private SqlListenerConnectionContext prepareContext(SqlListenerProtocolVersion ver, BinaryReaderExImpl reader) {
+        byte clientType = reader.readByte();
+
+        boolean distributedJoins = reader.readBoolean();
+        boolean enforceJoinOrder = reader.readBoolean();
+
+        SqlListenerRequestHandlerImpl handler = new SqlListenerRequestHandlerImpl(ctx, busyLock, maxCursors,
+            distributedJoins, enforceJoinOrder);
+
+        SqlListenerMessageParser parser = null;
+
+        switch (clientType) {
+            case ODBC_CLIENT:
+                parser = new OdbcMessageParser(ctx);
+
+                break;
+
+            case JDBC_CLIENT:
+                parser = new JdbcMessageParser(ctx);
+
+                break;
+        }
+
+        if (parser == null)
+            throw new IgniteException("Unknown client type: " + clientType);
+
+        return new SqlListenerConnectionContext(handler, parser);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
new file mode 100644
index 0000000..cbe54df
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
@@ -0,0 +1,191 @@
+/*
+ * 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.odbc;
+
+import java.net.InetAddress;
+import java.nio.ByteOrder;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.OdbcConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.HostAndPortRange;
+import org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter;
+import org.apache.ignite.internal.util.nio.GridNioCodecFilter;
+import org.apache.ignite.internal.util.nio.GridNioFilter;
+import org.apache.ignite.internal.util.nio.GridNioServer;
+import org.apache.ignite.internal.util.nio.GridNioSession;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.IgnitePortProtocol;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * ODBC processor.
+ */
+public class SqlListenerProcessor extends GridProcessorAdapter {
+    /** Default number of selectors. */
+    private static final int DFLT_SELECTOR_CNT = Math.min(4, Runtime.getRuntime().availableProcessors());
+
+    /** Default TCP_NODELAY flag. */
+    private static final boolean DFLT_TCP_NODELAY = true;
+
+    /** Default TCP direct buffer flag. */
+    private static final boolean DFLT_TCP_DIRECT_BUF = false;
+
+    /** Busy lock. */
+    private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
+
+    /** ODBC TCP Server. */
+    private GridNioServer<byte[]> srv;
+
+    /** ODBC executor service. */
+    private ExecutorService odbcExecSvc;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public SqlListenerProcessor(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start(boolean activeOnStart) throws IgniteCheckedException {
+        IgniteConfiguration cfg = ctx.config();
+
+        OdbcConfiguration odbcCfg = cfg.getOdbcConfiguration();
+
+        if (odbcCfg != null) {
+            try {
+                HostAndPortRange hostPort;
+
+                if (F.isEmpty(odbcCfg.getEndpointAddress())) {
+                    hostPort = new HostAndPortRange(OdbcConfiguration.DFLT_TCP_HOST,
+                        OdbcConfiguration.DFLT_TCP_PORT_FROM,
+                        OdbcConfiguration.DFLT_TCP_PORT_TO
+                    );
+                }
+                else {
+                    hostPort = HostAndPortRange.parse(odbcCfg.getEndpointAddress(),
+                        OdbcConfiguration.DFLT_TCP_PORT_FROM,
+                        OdbcConfiguration.DFLT_TCP_PORT_TO,
+                        "Failed to parse ODBC endpoint address"
+                    );
+                }
+
+                assertParameter(odbcCfg.getThreadPoolSize() > 0, "threadPoolSize > 0");
+
+                odbcExecSvc = new IgniteThreadPoolExecutor(
+                    "odbc",
+                    cfg.getIgniteInstanceName(),
+                    odbcCfg.getThreadPoolSize(),
+                    odbcCfg.getThreadPoolSize(),
+                    0,
+                    new LinkedBlockingQueue<Runnable>());
+
+                InetAddress host;
+
+                try {
+                    host = InetAddress.getByName(hostPort.host());
+                }
+                catch (Exception e) {
+                    throw new IgniteCheckedException("Failed to resolve ODBC host: " + hostPort.host(), e);
+                }
+
+                Exception lastErr = null;
+
+                for (int port = hostPort.portFrom(); port <= hostPort.portTo(); port++) {
+                    try {
+                        GridNioFilter[] filters = new GridNioFilter[] {
+                            new GridNioAsyncNotifyFilter(ctx.igniteInstanceName(), odbcExecSvc, log) {
+                                @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
+                                    proceedSessionOpened(ses);
+                                }
+                            },
+                            new GridNioCodecFilter(new SqlListenerBufferedParser(), log, false)
+                        };
+
+                        GridNioServer<byte[]> srv0 = GridNioServer.<byte[]>builder()
+                            .address(host)
+                            .port(port)
+                            .listener(new SqlListenerNioListener(ctx, busyLock, odbcCfg.getMaxOpenCursors()))
+                            .logger(log)
+                            .selectorCount(DFLT_SELECTOR_CNT)
+                            .igniteInstanceName(ctx.igniteInstanceName())
+                            .serverName("odbc")
+                            .tcpNoDelay(DFLT_TCP_NODELAY)
+                            .directBuffer(DFLT_TCP_DIRECT_BUF)
+                            .byteOrder(ByteOrder.nativeOrder())
+                            .socketSendBufferSize(odbcCfg.getSocketSendBufferSize())
+                            .socketReceiveBufferSize(odbcCfg.getSocketReceiveBufferSize())
+                            .filters(filters)
+                            .directMode(false)
+                            .build();
+
+                        srv0.start();
+
+                        srv = srv0;
+
+                        ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass());
+
+                        log.info("ODBC processor has started on TCP port " + port);
+
+                        lastErr = null;
+
+                        break;
+                    }
+                    catch (Exception e) {
+                        lastErr = e;
+                    }
+                }
+
+                assert (srv != null && lastErr == null) || (srv == null && lastErr != null);
+
+                if (lastErr != null)
+                    throw new IgniteCheckedException("Failed to bind to any [host:port] from the range [" +
+                        "address=" + hostPort + ", lastErr=" + lastErr + ']');
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to start ODBC processor.", e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (srv != null) {
+            busyLock.block();
+
+            srv.stop();
+
+            ctx.ports().deregisterPorts(getClass());
+
+            if (odbcExecSvc != null) {
+                U.shutdownNow(getClass(), odbcExecSvc, log);
+
+                odbcExecSvc = null;
+            }
+
+            if (log.isDebugEnabled())
+                log.debug("ODBC processor stopped.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandlerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandlerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandlerImpl.java
new file mode 100644
index 0000000..1230bb4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandlerImpl.java
@@ -0,0 +1,494 @@
+/*
+ * 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.odbc;
+
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+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.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_COLS;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_PARAMS;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_TBLS;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_CLOSE;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_EXEC;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_FETCH;
+
+/**
+ * SQL query handler.
+ */
+public class SqlListenerRequestHandlerImpl implements SqlListenerRequestHandler {
+    /** Query ID sequence. */
+    private static final AtomicLong QRY_ID_GEN = new AtomicLong();
+
+    /** Kernel context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Busy lock. */
+    private final GridSpinBusyLock busyLock;
+
+    /** Maximum allowed cursors. */
+    private final int maxCursors;
+
+    /** Current queries cursors. */
+    private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCursors = new ConcurrentHashMap<>();
+
+    /** Distributed joins flag. */
+    private final boolean distributedJoins;
+
+    /** Enforce join order flag. */
+    private final boolean enforceJoinOrder;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     * @param busyLock Shutdown latch.
+     * @param maxCursors Maximum allowed cursors.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce join order flag.
+     */
+    public SqlListenerRequestHandlerImpl(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors,
+        boolean distributedJoins, boolean enforceJoinOrder) {
+        this.ctx = ctx;
+        this.busyLock = busyLock;
+        this.maxCursors = maxCursors;
+        this.distributedJoins = distributedJoins;
+        this.enforceJoinOrder = enforceJoinOrder;
+
+        log = ctx.log(getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlListenerResponse handle(SqlListenerRequest req) {
+        assert req != null;
+
+        if (!busyLock.enterBusy())
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Failed to handle ODBC request because node is stopping: " + req);
+
+        try {
+            switch (req.command()) {
+                case QRY_EXEC:
+                    return executeQuery((SqlListenerQueryExecuteRequest)req);
+
+                case QRY_FETCH:
+                    return fetchQuery((SqlListenerQueryFetchRequest)req);
+
+                case QRY_CLOSE:
+                    return closeQuery((SqlListenerQueryCloseRequest)req);
+
+                case META_COLS:
+                    return getColumnsMeta((OdbcQueryGetColumnsMetaRequest)req);
+
+                case META_TBLS:
+                    return getTablesMeta((OdbcQueryGetTablesMetaRequest)req);
+
+                case META_PARAMS:
+                    return getParamsMeta((OdbcQueryGetParamsMetaRequest)req);
+            }
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * {@link SqlListenerQueryExecuteRequest} command handler.
+     *
+     * @param req Execute query request.
+     * @return Response.
+     */
+    private SqlListenerResponse executeQuery(SqlListenerQueryExecuteRequest req) {
+        int cursorCnt = qryCursors.size();
+
+        if (maxCursors > 0 && cursorCnt >= maxCursors)
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, "Too many opened cursors (either close other " +
+                "opened cursors or increase the limit through OdbcConfiguration.setMaxOpenCursors()) " +
+                "[maximum=" + maxCursors + ", current=" + cursorCnt + ']');
+
+        long qryId = QRY_ID_GEN.getAndIncrement();
+
+        try {
+            String sql = OdbcEscapeUtils.parse(req.sqlQuery());
+
+            if (log.isDebugEnabled())
+                log.debug("ODBC query parsed [reqId=" + req.requestId() + ", original=" + req.sqlQuery() +
+                    ", parsed=" + sql + ']');
+
+            SqlFieldsQuery qry = new SqlFieldsQuery(sql);
+
+            qry.setArgs(req.arguments());
+
+            qry.setDistributedJoins(distributedJoins);
+            qry.setEnforceJoinOrder(enforceJoinOrder);
+
+            IgniteCache<Object, Object> cache0 = ctx.grid().cache(req.cacheName());
+
+            if (cache0 == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
+
+            IgniteCache<Object, Object> cache = cache0.withKeepBinary();
+
+            if (cache == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Can not get cache with keep binary: " + req.cacheName());
+
+            QueryCursor qryCur = cache.query(qry);
+
+            qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
+
+            List<?> fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta();
+
+            SqlListenerQueryExecuteResult res = new SqlListenerQueryExecuteResult(qryId, convertMetadata(fieldsMeta));
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            qryCursors.remove(qryId);
+
+            U.error(log, "Failed to execute SQL query [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link SqlListenerQueryCloseRequest} command handler.
+     *
+     * @param req Execute query request.
+     * @return Response.
+     */
+    private SqlListenerResponse closeQuery(SqlListenerQueryCloseRequest req) {
+        try {
+            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
+
+            if (tuple == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Failed to find query with ID: " + req.queryId());
+
+            QueryCursor cur = tuple.get1();
+
+            assert(cur != null);
+
+            cur.close();
+
+            qryCursors.remove(req.queryId());
+
+            SqlListenerQueryCloseResult res = new SqlListenerQueryCloseResult(req.queryId());
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            qryCursors.remove(req.queryId());
+
+            U.error(log, "Failed to close SQL query [reqId=" + req.requestId() + ", req=" + req.queryId() + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link SqlListenerQueryFetchRequest} command handler.
+     *
+     * @param req Execute query request.
+     * @return Response.
+     */
+    private SqlListenerResponse fetchQuery(SqlListenerQueryFetchRequest req) {
+        try {
+            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
+
+            if (tuple == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Failed to find query with ID: " + req.queryId());
+
+            Iterator iter = tuple.get2();
+
+            if (iter == null) {
+                QueryCursor cur = tuple.get1();
+
+                iter = cur.iterator();
+
+                tuple.put(cur, iter);
+            }
+
+            List<Object> items = new ArrayList<>();
+
+            for (int i = 0; i < req.pageSize() && iter.hasNext(); ++i)
+                items.add(iter.next());
+
+            SqlListenerQueryFetchResult res = new SqlListenerQueryFetchResult(req.queryId(), items, !iter.hasNext());
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to fetch SQL query result [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetColumnsMetaRequest} command handler.
+     *
+     * @param req Get columns metadata request.
+     * @return Response.
+     */
+    private SqlListenerResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) {
+        try {
+            List<SqlListenerColumnMeta> meta = new ArrayList<>();
+
+            String cacheName;
+            String tableName;
+
+            if (req.tableName().contains(".")) {
+                // Parsing two-part table name.
+                String[] parts = req.tableName().split("\\.");
+
+                cacheName = OdbcUtils.removeQuotationMarksIfNeeded(parts[0]);
+
+                tableName = parts[1];
+            }
+            else {
+                cacheName = OdbcUtils.removeQuotationMarksIfNeeded(req.cacheName());
+
+                tableName = req.tableName();
+            }
+
+            Collection<GridQueryTypeDescriptor> tablesMeta = ctx.query().types(cacheName);
+
+            for (GridQueryTypeDescriptor table : tablesMeta) {
+                if (!matches(table.name(), tableName))
+                    continue;
+
+                for (Map.Entry<String, Class<?>> field : table.fields().entrySet()) {
+                    if (!matches(field.getKey(), req.columnName()))
+                        continue;
+
+                    SqlListenerColumnMeta columnMeta = new SqlListenerColumnMeta(req.cacheName(), table.name(),
+                        field.getKey(), field.getValue());
+
+                    if (!meta.contains(columnMeta))
+                        meta.add(columnMeta);
+                }
+            }
+
+            OdbcQueryGetColumnsMetaResult res = new OdbcQueryGetColumnsMetaResult(meta);
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get columns metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetTablesMetaRequest} command handler.
+     *
+     * @param req Get tables metadata request.
+     * @return Response.
+     */
+    private SqlListenerResponse getTablesMeta(OdbcQueryGetTablesMetaRequest req) {
+        try {
+            List<OdbcTableMeta> meta = new ArrayList<>();
+
+            String realSchema = OdbcUtils.removeQuotationMarksIfNeeded(req.schema());
+
+            for (String cacheName : ctx.cache().cacheNames())
+            {
+                if (!matches(cacheName, realSchema))
+                    continue;
+
+                Collection<GridQueryTypeDescriptor> tablesMeta = ctx.query().types(cacheName);
+
+                for (GridQueryTypeDescriptor table : tablesMeta) {
+                    if (!matches(table.name(), req.table()))
+                        continue;
+
+                    if (!matches("TABLE", req.tableType()))
+                        continue;
+
+                    OdbcTableMeta tableMeta = new OdbcTableMeta(null, cacheName, table.name(), "TABLE");
+
+                    if (!meta.contains(tableMeta))
+                        meta.add(tableMeta);
+                }
+            }
+
+            OdbcQueryGetTablesMetaResult res = new OdbcQueryGetTablesMetaResult(meta);
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get tables metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetParamsMetaRequest} command handler.
+     *
+     * @param req Get params metadata request.
+     * @return Response.
+     */
+    private SqlListenerResponse getParamsMeta(OdbcQueryGetParamsMetaRequest req) {
+        try {
+            PreparedStatement stmt = ctx.query().prepareNativeStatement(req.cacheName(), req.query());
+
+            ParameterMetaData pmd = stmt.getParameterMetaData();
+
+            byte[] typeIds = new byte[pmd.getParameterCount()];
+
+            for (int i = 1; i <= pmd.getParameterCount(); ++i) {
+                int sqlType = pmd.getParameterType(i);
+
+                typeIds[i - 1] = sqlTypeToBinary(sqlType);
+            }
+
+            OdbcQueryGetParamsMetaResult res = new OdbcQueryGetParamsMetaResult(typeIds);
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get params metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * Convert {@link java.sql.Types} to binary type constant (See {@link GridBinaryMarshaller} constants).
+     *
+     * @param sqlType SQL type.
+     * @return Binary type.
+     */
+    private static byte sqlTypeToBinary(int sqlType) {
+        switch (sqlType) {
+            case Types.BIGINT:
+                return GridBinaryMarshaller.LONG;
+
+            case Types.BOOLEAN:
+                return GridBinaryMarshaller.BOOLEAN;
+
+            case Types.DATE:
+                return GridBinaryMarshaller.DATE;
+
+            case Types.DOUBLE:
+                return GridBinaryMarshaller.DOUBLE;
+
+            case Types.FLOAT:
+            case Types.REAL:
+                return GridBinaryMarshaller.FLOAT;
+
+            case Types.NUMERIC:
+            case Types.DECIMAL:
+                return GridBinaryMarshaller.DECIMAL;
+
+            case Types.INTEGER:
+                return GridBinaryMarshaller.INT;
+
+            case Types.SMALLINT:
+                return GridBinaryMarshaller.SHORT;
+
+            case Types.TIME:
+                return GridBinaryMarshaller.TIME;
+
+            case Types.TIMESTAMP:
+                return GridBinaryMarshaller.TIMESTAMP;
+
+            case Types.TINYINT:
+                return GridBinaryMarshaller.BYTE;
+
+            case Types.CHAR:
+            case Types.VARCHAR:
+            case Types.LONGNVARCHAR:
+                return GridBinaryMarshaller.STRING;
+
+            case Types.NULL:
+                return GridBinaryMarshaller.NULL;
+
+            case Types.BINARY:
+            case Types.VARBINARY:
+            case Types.LONGVARBINARY:
+            default:
+                return GridBinaryMarshaller.BYTE_ARR;
+        }
+    }
+
+    /**
+     * Convert metadata in collection from {@link GridQueryFieldMetadata} to
+     * {@link SqlListenerColumnMeta}.
+     *
+     * @param meta Internal query field metadata.
+     * @return Odbc query field metadata.
+     */
+    private static Collection<SqlListenerColumnMeta> convertMetadata(Collection<?> meta) {
+        List<SqlListenerColumnMeta> res = new ArrayList<>();
+
+        if (meta != null) {
+            for (Object info : meta) {
+                assert info instanceof GridQueryFieldMetadata;
+
+                res.add(new SqlListenerColumnMeta((GridQueryFieldMetadata)info));
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * Checks whether string matches SQL pattern.
+     *
+     * @param str String.
+     * @param ptrn Pattern.
+     * @return Whether string matches pattern.
+     */
+    private static boolean matches(String str, String ptrn) {
+        return str != null && (F.isEmpty(ptrn) ||
+            str.toUpperCase().matches(ptrn.toUpperCase().replace("%", ".*").replace("_", ".")));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
new file mode 100644
index 0000000..cf87712
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
@@ -0,0 +1,50 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
+import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.processors.odbc.SqlListenerAbstractMessageParser;
+
+/**
+ * JDBC message parser.
+ */
+public class JdbcMessageParser extends SqlListenerAbstractMessageParser {
+    /**
+     * @param ctx Context.
+     */
+    public JdbcMessageParser(GridKernalContext ctx) {
+        super(ctx, new JdbcObjectReader(), new JdbcObjectWriter());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected BinaryReaderExImpl createReader(byte[] msg) {
+        BinaryInputStream stream = new BinaryHeapInputStream(msg);
+
+        return new BinaryReaderExImpl(null, stream, ctx.config().getClassLoader(), true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected BinaryWriterExImpl createWriter(int cap) {
+        return new BinaryWriterExImpl(null, new BinaryHeapOutputStream(cap), null, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectReader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectReader.java
new file mode 100644
index 0000000..81c8c10
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.processors.odbc.SqlListenerAbstractObjectReader;
+
+/**
+ * Binary reader with marshaling non-primitive and non-embedded objects with JDK marshaller.
+ */
+@SuppressWarnings("unchecked")
+public class JdbcObjectReader extends SqlListenerAbstractObjectReader {
+    /** {@inheritDoc} */
+    @Override protected Object readCustomObject(BinaryReaderExImpl reader) throws BinaryObjectException {
+        throw new BinaryObjectException("JDBC doesn't support custom objects.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectWriter.java
new file mode 100644
index 0000000..e87ef50
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcObjectWriter.java
@@ -0,0 +1,33 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.processors.odbc.SqlListenerAbstractObjectWriter;
+
+/**
+ * Binary writer with marshaling non-primitive and non-embedded objects with JDK marshaller..
+ */
+public class JdbcObjectWriter extends SqlListenerAbstractObjectWriter {
+    /** {@inheritDoc} */
+    @Override protected void writeCustomObject(BinaryWriterExImpl writer, Object obj)
+        throws BinaryObjectException {
+        throw new BinaryObjectException("JDBC doesn't support custom objects.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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 af595b9..300385f 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
@@ -18,262 +18,51 @@
 package org.apache.ignite.internal.processors.odbc.odbc;
 
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryThreadLocalContext;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 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.OdbcQueryGetColumnsMetaRequest;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaResult;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaRequest;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaResult;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaRequest;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaResult;
-import org.apache.ignite.internal.processors.odbc.OdbcTableMeta;
-import org.apache.ignite.internal.processors.odbc.SqlListenerColumnMeta;
-import org.apache.ignite.internal.processors.odbc.SqlListenerMessageParser;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseResult;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteResult;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchResult;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
-
-import java.util.Collection;
+import org.apache.ignite.internal.processors.odbc.SqlListenerAbstractMessageParser;
 
 /**
- * ODBC message parser.
+ * JDBC message parser.
  */
-public class OdbcMessageParser implements SqlListenerMessageParser {
-    /** Initial output stream capacity. */
-    private static final int INIT_CAP = 1024;
-
+public class OdbcMessageParser extends SqlListenerAbstractMessageParser {
     /** Marshaller. */
     private final GridBinaryMarshaller marsh;
 
-    /** Logger. */
-    private final IgniteLogger log;
-
     /**
      * @param ctx Context.
      */
-    public OdbcMessageParser(final GridKernalContext ctx) {
-        CacheObjectBinaryProcessorImpl cacheObjProc = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects();
+    public OdbcMessageParser(GridKernalContext ctx) {
+        super(ctx, new OdbcObjectReader(), new OdbcObjectWriter());
 
-        this.marsh = cacheObjProc.marshaller();
+        if (ctx.cacheObjects() instanceof CacheObjectBinaryProcessorImpl) {
+            CacheObjectBinaryProcessorImpl cacheObjProc = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects();
 
-        this.log = ctx.log(getClass());
+            marsh = cacheObjProc.marshaller();
+        }
+        else {
+            throw new IgniteException("ODBC can only be used with BinaryMarshaller (please set it " +
+                "through IgniteConfiguration.setMarshaller())");
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerRequest decode(byte[] msg) {
-        assert msg != null;
-
+    @Override protected BinaryReaderExImpl createReader(byte[] msg) {
         BinaryInputStream stream = new BinaryHeapInputStream(msg);
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null, true);
-
-        byte cmd = reader.readByte();
-
-        SqlListenerRequest res;
-
-        switch (cmd) {
-            case SqlListenerRequest.QRY_EXEC: {
-                String cache = reader.readString();
-                String sql = reader.readString();
-                int argsNum = reader.readInt();
-
-                Object[] params = new Object[argsNum];
-
-                for (int i = 0; i < argsNum; ++i)
-                    params[i] = reader.readObjectDetached();
-
-                res = new SqlListenerQueryExecuteRequest(cache, sql, params);
-
-                break;
-            }
-
-            case SqlListenerRequest.QRY_FETCH: {
-                long queryId = reader.readLong();
-                int pageSize = reader.readInt();
-
-                res = new SqlListenerQueryFetchRequest(queryId, pageSize);
-
-                break;
-            }
-
-            case SqlListenerRequest.QRY_CLOSE: {
-                long queryId = reader.readLong();
-
-                res = new SqlListenerQueryCloseRequest(queryId);
-
-                break;
-            }
-
-            case SqlListenerRequest.META_COLS: {
-                String cache = reader.readString();
-                String table = reader.readString();
-                String column = reader.readString();
-
-                res = new OdbcQueryGetColumnsMetaRequest(cache, table, column);
-
-                break;
-            }
-
-            case SqlListenerRequest.META_TBLS: {
-                String catalog = reader.readString();
-                String schema = reader.readString();
-                String table = reader.readString();
-                String tableType = reader.readString();
-
-                res = new OdbcQueryGetTablesMetaRequest(catalog, schema, table, tableType);
-
-                break;
-            }
-
-            case SqlListenerRequest.META_PARAMS: {
-                String cacheName = reader.readString();
-                String sqlQuery = reader.readString();
-
-                res = new OdbcQueryGetParamsMetaRequest(cacheName, sqlQuery);
-
-                break;
-            }
-
-            default:
-                throw new IgniteException("Unknown ODBC command: [cmd=" + cmd + ']');
-        }
-
-        return res;
+        return new BinaryReaderExImpl(marsh.context(), stream, ctx.config().getClassLoader(), true);
     }
 
     /** {@inheritDoc} */
-    @Override public byte[] encode(SqlListenerResponse msg) {
-        assert msg != null;
-
-        // Creating new binary writer
-        BinaryWriterExImpl writer = marsh.writer(new BinaryHeapOutputStream(INIT_CAP));
-
-        // Writing status.
-        writer.writeByte((byte) msg.status());
-
-        if (msg.status() != SqlListenerResponse.STATUS_SUCCESS) {
-            writer.writeString(msg.error());
-
-            return writer.array();
-        }
-
-        Object res0 = msg.response();
-
-        if (res0 == null)
-            return writer.array();
-        else if (res0 instanceof SqlListenerQueryExecuteResult) {
-            SqlListenerQueryExecuteResult res = (SqlListenerQueryExecuteResult) res0;
-
-            if (log.isDebugEnabled())
-                log.debug("Resulting query ID: " + res.getQueryId());
-
-            writer.writeLong(res.getQueryId());
-
-            Collection<SqlListenerColumnMeta> metas = res.getColumnsMetadata();
-
-            assert metas != null;
-
-            writer.writeInt(metas.size());
-
-            for (SqlListenerColumnMeta meta : metas)
-                meta.write(writer);
-        }
-        else if (res0 instanceof SqlListenerQueryFetchResult) {
-            SqlListenerQueryFetchResult res = (SqlListenerQueryFetchResult) res0;
-
-            if (log.isDebugEnabled())
-                log.debug("Resulting query ID: " + res.queryId());
-
-            writer.writeLong(res.queryId());
-
-            Collection<?> items0 = res.items();
-
-            assert items0 != null;
-
-            writer.writeBoolean(res.last());
-
-            writer.writeInt(items0.size());
-
-            for (Object row0 : items0) {
-                if (row0 != null) {
-                    Collection<?> row = (Collection<?>)row0;
-
-                    writer.writeInt(row.size());
-
-                    for (Object obj : row) {
-                        if (obj == null) {
-                            writer.writeObjectDetached(null);
-                            continue;
-                        }
-
-                        Class<?> cls = obj.getClass();
-
-                        if (cls == java.sql.Time.class)
-                            writer.writeTime((java.sql.Time)obj);
-                        else if (cls == java.sql.Timestamp.class)
-                            writer.writeTimestamp((java.sql.Timestamp)obj);
-                        else if (cls == java.sql.Date.class)
-                            writer.writeDate((java.util.Date)obj);
-                        else
-                            writer.writeObjectDetached(obj);
-                    }
-                }
-            }
-        }
-        else if (res0 instanceof SqlListenerQueryCloseResult) {
-            SqlListenerQueryCloseResult res = (SqlListenerQueryCloseResult) res0;
-
-            if (log.isDebugEnabled())
-                log.debug("Resulting query ID: " + res.getQueryId());
-
-            writer.writeLong(res.getQueryId());
-        }
-        else if (res0 instanceof OdbcQueryGetColumnsMetaResult) {
-            OdbcQueryGetColumnsMetaResult res = (OdbcQueryGetColumnsMetaResult) res0;
-
-            Collection<SqlListenerColumnMeta> columnsMeta = res.meta();
-
-            assert columnsMeta != null;
-
-            writer.writeInt(columnsMeta.size());
-
-            for (SqlListenerColumnMeta columnMeta : columnsMeta)
-                columnMeta.write(writer);
-        }
-        else if (res0 instanceof OdbcQueryGetTablesMetaResult) {
-            OdbcQueryGetTablesMetaResult res = (OdbcQueryGetTablesMetaResult) res0;
-
-            Collection<OdbcTableMeta> tablesMeta = res.meta();
-
-            assert tablesMeta != null;
-
-            writer.writeInt(tablesMeta.size());
-
-            for (OdbcTableMeta tableMeta : tablesMeta)
-                tableMeta.writeBinary(writer);
-        }
-        else if (res0 instanceof OdbcQueryGetParamsMetaResult) {
-            OdbcQueryGetParamsMetaResult res = (OdbcQueryGetParamsMetaResult) res0;
-
-            byte[] typeIds = res.typeIds();
-
-            writer.writeObjectDetached(typeIds);
-        }
-        else
-            assert false : "Should not reach here.";
-
-        return writer.array();
+    @Override protected BinaryWriterExImpl createWriter(int cap) {
+        return new BinaryWriterExImpl(marsh.context(), new BinaryHeapOutputStream(cap),
+            BinaryThreadLocalContext.get().schemaHolder(), null);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectReader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectReader.java
new file mode 100644
index 0000000..586fbc5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.odbc.odbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.processors.odbc.SqlListenerAbstractObjectReader;
+
+/**
+ * Binary reader with marshaling non-primitive and non-embedded objects with JDK marshaller.
+ */
+@SuppressWarnings("unchecked")
+public class OdbcObjectReader extends SqlListenerAbstractObjectReader {
+    /** {@inheritDoc} */
+    @Override protected Object readCustomObject(BinaryReaderExImpl reader) throws BinaryObjectException {
+        return reader.readObjectDetached();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectWriter.java
new file mode 100644
index 0000000..c2f3aba
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcObjectWriter.java
@@ -0,0 +1,32 @@
+/*
+ * 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.odbc.odbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.processors.odbc.SqlListenerAbstractObjectWriter;
+
+/**
+ * Binary writer with marshaling non-primitive and non-embedded objects with JDK marshaller..
+ */
+public class OdbcObjectWriter extends SqlListenerAbstractObjectWriter {
+    /** {@inheritDoc} */
+    @Override protected void writeCustomObject(BinaryWriterExImpl writer, Object obj) throws BinaryObjectException {
+        writer.writeObjectDetached(obj);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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
deleted file mode 100644
index eabc486..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ /dev/null
@@ -1,513 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.odbc.odbc;
-
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.binary.GridBinaryMarshaller;
-import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaRequest;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaResult;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaRequest;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaResult;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaRequest;
-import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaResult;
-import org.apache.ignite.internal.processors.odbc.OdbcTableMeta;
-import org.apache.ignite.internal.processors.odbc.OdbcUtils;
-import org.apache.ignite.internal.processors.odbc.SqlListenerColumnMeta;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseResult;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteResult;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchResult;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
-import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
-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.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
-
-import java.sql.ParameterMetaData;
-import java.sql.PreparedStatement;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_COLS;
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_PARAMS;
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_TBLS;
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_CLOSE;
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_EXEC;
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_FETCH;
-
-/**
- * SQL query handler.
- */
-public class OdbcRequestHandler implements SqlListenerRequestHandler {
-    /** Query ID sequence. */
-    private static final AtomicLong QRY_ID_GEN = new AtomicLong();
-
-    /** Kernel context. */
-    private final GridKernalContext ctx;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /** Busy lock. */
-    private final GridSpinBusyLock busyLock;
-
-    /** Maximum allowed cursors. */
-    private final int maxCursors;
-
-    /** Current queries cursors. */
-    private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCursors = new ConcurrentHashMap<>();
-
-    /** Distributed joins flag. */
-    private final boolean distributedJoins;
-
-    /** Enforce join order flag. */
-    private final boolean enforceJoinOrder;
-
-    /**
-     * Constructor.
-     *
-     * @param ctx Context.
-     * @param busyLock Shutdown latch.
-     * @param maxCursors Maximum allowed cursors.
-     * @param distributedJoins Distributed joins flag.
-     * @param enforceJoinOrder Enforce join order flag.
-     */
-    public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors,
-        boolean distributedJoins, boolean enforceJoinOrder) {
-        this.ctx = ctx;
-        this.busyLock = busyLock;
-        this.maxCursors = maxCursors;
-        this.distributedJoins = distributedJoins;
-        this.enforceJoinOrder = enforceJoinOrder;
-
-        log = ctx.log(getClass());
-    }
-
-    /** {@inheritDoc} */
-    @Override public SqlListenerResponse handle(SqlListenerRequest req) {
-        assert req != null;
-
-        if (!busyLock.enterBusy())
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
-                    "Failed to handle ODBC request because node is stopping: " + req);
-
-        try {
-            switch (req.command()) {
-                case QRY_EXEC:
-                    return executeQuery((SqlListenerQueryExecuteRequest)req);
-
-                case QRY_FETCH:
-                    return fetchQuery((SqlListenerQueryFetchRequest)req);
-
-                case QRY_CLOSE:
-                    return closeQuery((SqlListenerQueryCloseRequest)req);
-
-                case META_COLS:
-                    return getColumnsMeta((OdbcQueryGetColumnsMetaRequest)req);
-
-                case META_TBLS:
-                    return getTablesMeta((OdbcQueryGetTablesMetaRequest)req);
-
-                case META_PARAMS:
-                    return getParamsMeta((OdbcQueryGetParamsMetaRequest)req);
-            }
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
-     * {@link SqlListenerQueryExecuteRequest} command handler.
-     *
-     * @param req Execute query request.
-     * @return Response.
-     */
-    private SqlListenerResponse executeQuery(SqlListenerQueryExecuteRequest req) {
-        int cursorCnt = qryCursors.size();
-
-        if (maxCursors > 0 && cursorCnt >= maxCursors)
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, "Too many opened cursors (either close other " +
-                "opened cursors or increase the limit through OdbcConfiguration.setMaxOpenCursors()) " +
-                "[maximum=" + maxCursors + ", current=" + cursorCnt + ']');
-
-        long qryId = QRY_ID_GEN.getAndIncrement();
-
-        try {
-            String sql = OdbcEscapeUtils.parse(req.sqlQuery());
-
-            if (log.isDebugEnabled())
-                log.debug("ODBC query parsed [reqId=" + req.requestId() + ", original=" + req.sqlQuery() +
-                    ", parsed=" + sql + ']');
-
-            SqlFieldsQuery qry = new SqlFieldsQuery(sql);
-
-            qry.setArgs(req.arguments());
-
-            qry.setDistributedJoins(distributedJoins);
-            qry.setEnforceJoinOrder(enforceJoinOrder);
-
-            IgniteCache<Object, Object> cache0 = ctx.grid().cache(req.cacheName());
-
-            if (cache0 == null)
-                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
-                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
-
-            IgniteCache<Object, Object> cache = cache0.withKeepBinary();
-
-            if (cache == null)
-                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
-                    "Can not get cache with keep binary: " + req.cacheName());
-
-            QueryCursor qryCur = cache.query(qry);
-
-            qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
-
-            List<?> fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta();
-
-            SqlListenerQueryExecuteResult res = new SqlListenerQueryExecuteResult(qryId, convertMetadata(fieldsMeta));
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            qryCursors.remove(qryId);
-
-            U.error(log, "Failed to execute SQL query [reqId=" + req.requestId() + ", req=" + req + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link SqlListenerQueryCloseRequest} command handler.
-     *
-     * @param req Execute query request.
-     * @return Response.
-     */
-    private SqlListenerResponse closeQuery(SqlListenerQueryCloseRequest req) {
-        try {
-            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
-
-            if (tuple == null)
-                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
-                    "Failed to find query with ID: " + req.queryId());
-
-            QueryCursor cur = tuple.get1();
-
-            assert(cur != null);
-
-            cur.close();
-
-            qryCursors.remove(req.queryId());
-
-            SqlListenerQueryCloseResult res = new SqlListenerQueryCloseResult(req.queryId());
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            qryCursors.remove(req.queryId());
-
-            U.error(log, "Failed to close SQL query [reqId=" + req.requestId() + ", req=" + req.queryId() + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link SqlListenerQueryFetchRequest} command handler.
-     *
-     * @param req Execute query request.
-     * @return Response.
-     */
-    private SqlListenerResponse fetchQuery(SqlListenerQueryFetchRequest req) {
-        try {
-            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
-
-            if (tuple == null)
-                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
-                    "Failed to find query with ID: " + req.queryId());
-
-            Iterator iter = tuple.get2();
-
-            if (iter == null) {
-                QueryCursor cur = tuple.get1();
-
-                iter = cur.iterator();
-
-                tuple.put(cur, iter);
-            }
-
-            List<Object> items = new ArrayList<>();
-
-            for (int i = 0; i < req.pageSize() && iter.hasNext(); ++i)
-                items.add(iter.next());
-
-            SqlListenerQueryFetchResult res = new SqlListenerQueryFetchResult(req.queryId(), items, !iter.hasNext());
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to fetch SQL query result [reqId=" + req.requestId() + ", req=" + req + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link OdbcQueryGetColumnsMetaRequest} command handler.
-     *
-     * @param req Get columns metadata request.
-     * @return Response.
-     */
-    private SqlListenerResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) {
-        try {
-            List<SqlListenerColumnMeta> meta = new ArrayList<>();
-
-            String cacheName;
-            String tableName;
-
-            if (req.tableName().contains(".")) {
-                // Parsing two-part table name.
-                String[] parts = req.tableName().split("\\.");
-
-                cacheName = OdbcUtils.removeQuotationMarksIfNeeded(parts[0]);
-
-                tableName = parts[1];
-            }
-            else {
-                cacheName = OdbcUtils.removeQuotationMarksIfNeeded(req.cacheName());
-
-                tableName = req.tableName();
-            }
-
-            Collection<GridQueryTypeDescriptor> tablesMeta = ctx.query().types(cacheName);
-
-            for (GridQueryTypeDescriptor table : tablesMeta) {
-                if (!matches(table.name(), tableName))
-                    continue;
-
-                for (Map.Entry<String, Class<?>> field : table.fields().entrySet()) {
-                    if (!matches(field.getKey(), req.columnName()))
-                        continue;
-
-                    SqlListenerColumnMeta columnMeta = new SqlListenerColumnMeta(req.cacheName(), table.name(),
-                        field.getKey(), field.getValue());
-
-                    if (!meta.contains(columnMeta))
-                        meta.add(columnMeta);
-                }
-            }
-
-            OdbcQueryGetColumnsMetaResult res = new OdbcQueryGetColumnsMetaResult(meta);
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to get columns metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link OdbcQueryGetTablesMetaRequest} command handler.
-     *
-     * @param req Get tables metadata request.
-     * @return Response.
-     */
-    private SqlListenerResponse getTablesMeta(OdbcQueryGetTablesMetaRequest req) {
-        try {
-            List<OdbcTableMeta> meta = new ArrayList<>();
-
-            String realSchema = OdbcUtils.removeQuotationMarksIfNeeded(req.schema());
-
-            for (String cacheName : ctx.cache().cacheNames())
-            {
-                if (!matches(cacheName, realSchema))
-                    continue;
-
-                Collection<GridQueryTypeDescriptor> tablesMeta = ctx.query().types(cacheName);
-
-                for (GridQueryTypeDescriptor table : tablesMeta) {
-                    if (!matches(table.name(), req.table()))
-                        continue;
-
-                    if (!matches("TABLE", req.tableType()))
-                        continue;
-
-                    OdbcTableMeta tableMeta = new OdbcTableMeta(null, cacheName, table.name(), "TABLE");
-
-                    if (!meta.contains(tableMeta))
-                        meta.add(tableMeta);
-                }
-            }
-
-            OdbcQueryGetTablesMetaResult res = new OdbcQueryGetTablesMetaResult(meta);
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to get tables metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link OdbcQueryGetParamsMetaRequest} command handler.
-     *
-     * @param req Get params metadata request.
-     * @return Response.
-     */
-    private SqlListenerResponse getParamsMeta(OdbcQueryGetParamsMetaRequest req) {
-        try {
-            PreparedStatement stmt = ctx.query().prepareNativeStatement(req.cacheName(), req.query());
-
-            ParameterMetaData pmd = stmt.getParameterMetaData();
-
-            byte[] typeIds = new byte[pmd.getParameterCount()];
-
-            for (int i = 1; i <= pmd.getParameterCount(); ++i) {
-                int sqlType = pmd.getParameterType(i);
-
-                typeIds[i - 1] = sqlTypeToBinary(sqlType);
-            }
-
-            OdbcQueryGetParamsMetaResult res = new OdbcQueryGetParamsMetaResult(typeIds);
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to get params metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * Convert {@link java.sql.Types} to binary type constant (See {@link GridBinaryMarshaller} constants).
-     *
-     * @param sqlType SQL type.
-     * @return Binary type.
-     */
-    private static byte sqlTypeToBinary(int sqlType) {
-        switch (sqlType) {
-            case Types.BIGINT:
-                return GridBinaryMarshaller.LONG;
-
-            case Types.BOOLEAN:
-                return GridBinaryMarshaller.BOOLEAN;
-
-            case Types.DATE:
-                return GridBinaryMarshaller.DATE;
-
-            case Types.DOUBLE:
-                return GridBinaryMarshaller.DOUBLE;
-
-            case Types.FLOAT:
-            case Types.REAL:
-                return GridBinaryMarshaller.FLOAT;
-
-            case Types.NUMERIC:
-            case Types.DECIMAL:
-                return GridBinaryMarshaller.DECIMAL;
-
-            case Types.INTEGER:
-                return GridBinaryMarshaller.INT;
-
-            case Types.SMALLINT:
-                return GridBinaryMarshaller.SHORT;
-
-            case Types.TIME:
-                return GridBinaryMarshaller.TIME;
-
-            case Types.TIMESTAMP:
-                return GridBinaryMarshaller.TIMESTAMP;
-
-            case Types.TINYINT:
-                return GridBinaryMarshaller.BYTE;
-
-            case Types.CHAR:
-            case Types.VARCHAR:
-            case Types.LONGNVARCHAR:
-                return GridBinaryMarshaller.STRING;
-
-            case Types.NULL:
-                return GridBinaryMarshaller.NULL;
-
-            case Types.BINARY:
-            case Types.VARBINARY:
-            case Types.LONGVARBINARY:
-            default:
-                return GridBinaryMarshaller.BYTE_ARR;
-        }
-    }
-
-    /**
-     * Convert metadata in collection from {@link GridQueryFieldMetadata} to
-     * {@link SqlListenerColumnMeta}.
-     *
-     * @param meta Internal query field metadata.
-     * @return Odbc query field metadata.
-     */
-    private static Collection<SqlListenerColumnMeta> convertMetadata(Collection<?> meta) {
-        List<SqlListenerColumnMeta> res = new ArrayList<>();
-
-        if (meta != null) {
-            for (Object info : meta) {
-                assert info instanceof GridQueryFieldMetadata;
-
-                res.add(new SqlListenerColumnMeta((GridQueryFieldMetadata)info));
-            }
-        }
-
-        return res;
-    }
-
-    /**
-     * Checks whether string matches SQL pattern.
-     *
-     * @param str String.
-     * @param ptrn Pattern.
-     * @return Whether string matches pattern.
-     */
-    private static boolean matches(String str, String ptrn) {
-        return str != null && (F.isEmpty(ptrn) ||
-            str.toUpperCase().matches(ptrn.toUpperCase().replace("%", ".*").replace("_", ".")));
-    }
-}


[05/39] ignite git commit: IGNITE-5281: Indexing: changed "space" to "cacheName". No more "spaces". This closes #1992.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 c099d77..0ce905b 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
@@ -228,10 +228,7 @@ import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType
 
 /**
  * Indexing implementation based on H2 database engine. In this implementation main query language is SQL,
- * fulltext indexing can be performed using Lucene. For each registered space
- * the SPI will create respective schema, for default space (where space name is null) schema
- * with name {@code ""} will be used. To avoid name conflicts user should not explicitly name
- * a schema {@code ""}.
+ * fulltext indexing can be performed using Lucene.
  * <p>
  * For each registered {@link GridQueryTypeDescriptor} this SPI will create respective SQL table with
  * {@code '_key'} and {@code '_val'} fields for key and value, and fields from
@@ -336,8 +333,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** */
     private GridReduceQueryExecutor rdcQryExec;
 
-    /** space name -> schema name */
-    private final Map<String, String> space2schema = new ConcurrentHashMap8<>();
+    /** Cache name -> schema name */
+    private final Map<String, String> cacheName2schema = new ConcurrentHashMap8<>();
 
     /** */
     private AtomicLong qryIdGen;
@@ -429,12 +426,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @return Connection.
      */
-    public Connection connectionForSpace(String space) {
+    public Connection connectionForCache(String cacheName) {
         try {
-            return connectionForThread(schema(space));
+            return connectionForThread(schema(cacheName));
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);
@@ -484,13 +481,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException {
-        return prepareStatement(connectionForSpace(space), sql, true);
+    @Override public PreparedStatement prepareNativeStatement(String cacheName, String sql) throws SQLException {
+        return prepareStatement(connectionForCache(cacheName), sql, true);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteDataStreamer<?, ?> createStreamer(String spaceName, PreparedStatement nativeStmt,
+    @Override public IgniteDataStreamer<?, ?> createStreamer(String cacheName, PreparedStatement nativeStmt,
         long autoFlushFreq, int nodeBufSize, int nodeParOps, boolean allowOverwrite) {
         Prepared prep = GridSqlQueryParser.prepared(nativeStmt);
 
@@ -498,7 +495,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             throw new IgniteSQLException("Only INSERT operations are supported in streaming mode",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
-        IgniteDataStreamer streamer = ctx.grid().dataStreamer(spaceName);
+        IgniteDataStreamer streamer = ctx.grid().dataStreamer(cacheName);
 
         streamer.autoFlushFrequency(autoFlushFreq);
 
@@ -640,7 +637,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void store(String spaceName,
+    @Override public void store(String cacheName,
         String typeName,
         KeyCacheObject k,
         int partId,
@@ -648,7 +645,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridCacheVersion ver,
         long expirationTime,
         long link) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
+        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl == null)
             return; // Type was rejected.
@@ -674,29 +671,29 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @return Cache object context.
      */
-    private CacheObjectContext objectContext(String space) {
+    private CacheObjectContext objectContext(String cacheName) {
         if (ctx == null)
             return null;
 
-        return ctx.cache().internalCache(space).context().cacheObjectContext();
+        return ctx.cache().internalCache(cacheName).context().cacheObjectContext();
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @return Cache object context.
      */
-    private GridCacheContext cacheContext(String space) {
+    private GridCacheContext cacheContext(String cacheName) {
         if (ctx == null)
             return null;
 
-        return ctx.cache().internalCache(space).context();
+        return ctx.cache().internalCache(cacheName).context();
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(String spaceName,
+    @Override public void remove(String cacheName,
         GridQueryTypeDescriptor type,
         KeyCacheObject key,
         int partId,
@@ -705,7 +702,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']');
 
-        TableDescriptor tbl = tableDescriptor(type.name(), spaceName);
+        TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
 
         if (tbl == null)
             return;
@@ -764,12 +761,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Add initial user index.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param desc Table descriptor.
      * @param h2Idx User index.
      * @throws IgniteCheckedException If failed.
      */
-    private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2IndexBase h2Idx)
+    private void addInitialUserIndex(String cacheName, TableDescriptor desc, GridH2IndexBase h2Idx)
         throws IgniteCheckedException {
         GridH2Table h2Tbl = desc.tbl;
 
@@ -778,7 +775,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             String sql = indexCreateSql(desc.fullTableName(), h2Idx, false, desc.schema.escapeAll());
 
-            executeSql(spaceName, sql);
+            executeSql(cacheName, sql);
         }
         catch (Exception e) {
             // Rollback and re-throw.
@@ -789,11 +786,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void dynamicIndexCreate(final String spaceName, final String tblName,
+    @Override public void dynamicIndexCreate(final String cacheName, final String tblName,
         final QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor)
         throws IgniteCheckedException {
         // Locate table.
-        String schemaName = schema(spaceName);
+        String schemaName = schema(cacheName);
 
         Schema schema = schemas.get(schemaName);
 
@@ -834,7 +831,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             // prepared statements are re-built.
             String sql = indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists, schema.escapeAll());
 
-            executeSql(spaceName, sql);
+            executeSql(cacheName, sql);
         }
         catch (Exception e) {
             // Rollback and re-throw.
@@ -846,27 +843,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-    @Override public void dynamicIndexDrop(final String spaceName, String idxName, boolean ifExists)
+    @Override public void dynamicIndexDrop(final String cacheName, String idxName, boolean ifExists)
         throws IgniteCheckedException{
-        String schemaName = schema(spaceName);
+        String schemaName = schema(cacheName);
 
         Schema schema = schemas.get(schemaName);
 
         String sql = indexDropSql(schemaName, idxName, ifExists, schema.escapeAll());
 
-        executeSql(spaceName, sql);
+        executeSql(cacheName, sql);
     }
 
     /**
      * Execute DDL command.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param sql SQL.
      * @throws IgniteCheckedException If failed.
      */
-    private void executeSql(String spaceName, String sql) throws IgniteCheckedException {
+    private void executeSql(String cacheName, String sql) throws IgniteCheckedException {
         try {
-            Connection conn = connectionForSpace(spaceName);
+            Connection conn = connectionForCache(cacheName);
 
             try (PreparedStatement stmt = prepareStatement(conn, sql, false)) {
                 stmt.execute();
@@ -984,12 +981,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     @SuppressWarnings("unchecked")
     @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(
-        String spaceName, String qry, String typeName,
+        String cacheName, String qry, String typeName,
         IndexingQueryFilter filters) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
+        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl != null && tbl.luceneIdx != null) {
-            GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, spaceName,
+            GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, cacheName,
                 U.currentTimeMillis(), null, true);
 
             try {
@@ -1006,9 +1003,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void unregisterType(String spaceName, String typeName)
+    @Override public void unregisterType(String cacheName, String typeName)
         throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
+        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl != null)
             removeTable(tbl);
@@ -1017,10 +1014,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Queries individual fields (generally used by JDBC drivers).
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param params Query parameters.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @param enforceJoinOrder Enforce join order of tables in the query.
      * @param timeout Query timeout in milliseconds.
      * @param cancel Query cancel.
@@ -1028,11 +1025,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public GridQueryFieldsResult queryLocalSqlFields(final String spaceName, final String qry,
+    public GridQueryFieldsResult queryLocalSqlFields(final String cacheName, final String qry,
         @Nullable final Collection<Object> params, final IndexingQueryFilter filter, boolean enforceJoinOrder,
         final int timeout, final GridQueryCancel cancel)
         throws IgniteCheckedException {
-        final Connection conn = connectionForSpace(spaceName);
+        final Connection conn = connectionForCache(cacheName);
 
         setupConnection(conn, false, enforceJoinOrder);
 
@@ -1049,7 +1046,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             fldsQry.setEnforceJoinOrder(enforceJoinOrder);
             fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
 
-            return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filter, cancel);
+            return dmlProc.updateLocalSqlFields(cacheName, stmt, fldsQry, filter, cancel);
         }
         else if (DdlStatementsProcessor.isDdlStatement(p))
             throw new IgniteSQLException("DDL statements are supported for the whole cluster only",
@@ -1074,12 +1071,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 GridH2QueryContext.set(ctx);
 
                 GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL_FIELDS,
-                    spaceName, U.currentTimeMillis(), cancel, true);
+                    cacheName, U.currentTimeMillis(), cancel, true);
 
                 runs.putIfAbsent(run.id(), run);
 
                 try {
-                    ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
+                    ResultSet rs = executeSqlQueryWithTimer(cacheName, stmt, conn, qry, params, timeout, cancel);
 
                     return new FieldsIterator(rs);
                 }
@@ -1093,9 +1090,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public long streamUpdateQuery(String spaceName, String qry,
+    @Override public long streamUpdateQuery(String cacheName, String qry,
         @Nullable Object[] params, IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException {
-        final Connection conn = connectionForSpace(spaceName);
+        final Connection conn = connectionForCache(cacheName);
 
         final PreparedStatement stmt;
 
@@ -1235,7 +1232,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Executes sql query and prints warning if query is too slow..
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param conn Connection,.
      * @param sql Sql query.
      * @param params Parameters.
@@ -1244,21 +1241,21 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public ResultSet executeSqlQueryWithTimer(String space,
+    public ResultSet executeSqlQueryWithTimer(String cacheName,
         Connection conn,
         String sql,
         @Nullable Collection<Object> params,
         boolean useStmtCache,
         int timeoutMillis,
         @Nullable GridQueryCancel cancel) throws IgniteCheckedException {
-        return executeSqlQueryWithTimer(space, preparedStatementWithParams(conn, sql, params, useStmtCache),
+        return executeSqlQueryWithTimer(cacheName, preparedStatementWithParams(conn, sql, params, useStmtCache),
             conn, sql, params, timeoutMillis, cancel);
     }
 
     /**
      * Executes sql query and prints warning if query is too slow.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param stmt Prepared statement for query.
      * @param conn Connection.
      * @param sql Sql query.
@@ -1267,7 +1264,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    private ResultSet executeSqlQueryWithTimer(String space, PreparedStatement stmt,
+    private ResultSet executeSqlQueryWithTimer(String cacheName, PreparedStatement stmt,
         Connection conn,
         String sql,
         @Nullable Collection<Object> params,
@@ -1280,7 +1277,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             long time = U.currentTimeMillis() - start;
 
-            long longQryExecTimeout = schemas.get(schema(space)).ccfg.getLongQueryWarningTimeout();
+            long longQryExecTimeout = schemas.get(schema(cacheName)).ccfg.getLongQueryWarningTimeout();
 
             if (time > longQryExecTimeout) {
                 String msg = "Query execution is too long (" + time + " ms): " + sql;
@@ -1350,11 +1347,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         else {
             final boolean keepBinary = cctx.keepBinary();
 
-            final String space = cctx.name();
+            final String cacheName = cctx.name();
             final String sql = qry.getSql();
             final Object[] args = qry.getArgs();
 
-            final GridQueryFieldsResult res = queryLocalSqlFields(space, sql, F.asList(args), filter,
+            final GridQueryFieldsResult res = queryLocalSqlFields(cacheName, sql, F.asList(args), filter,
                 qry.isEnforceJoinOrder(), qry.getTimeout(), cancel);
 
             QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(new Iterable<List<?>>() {
@@ -1385,7 +1382,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return queryDistributedSql(cctx, qry);
         }
         else {
-            String space = cctx.name();
+            String cacheName = cctx.name();
             String type = qry.getType();
             String sqlQry = qry.getSql();
             String alias = qry.getAlias();
@@ -1393,7 +1390,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             GridQueryCancel cancel = new GridQueryCancel();
 
-            final GridCloseableIterator<IgniteBiTuple<K, V>> i = queryLocalSql(space, sqlQry, alias,
+            final GridCloseableIterator<IgniteBiTuple<K, V>> i = queryLocalSql(cacheName, sqlQry, alias,
                 F.asList(params), type, filter, cancel);
 
             return new QueryCursorImpl<Cache.Entry<K, V>>(new Iterable<Cache.Entry<K, V>>() {
@@ -1427,19 +1424,19 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Executes regular query.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param alias Table alias.
      * @param params Query parameters.
      * @param type Query return type.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @return Queried rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(String spaceName,
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(String cacheName,
         final String qry, String alias, @Nullable final Collection<Object> params, String type,
         final IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException {
-        final TableDescriptor tbl = tableDescriptor(type, spaceName);
+        final TableDescriptor tbl = tableDescriptor(type, cacheName);
 
         if (tbl == null)
             throw new IgniteSQLException("Failed to find SQL table for type: " + type,
@@ -1454,13 +1451,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter)
             .distributedJoinMode(OFF));
 
-        GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, spaceName,
+        GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, cacheName,
             U.currentTimeMillis(), null, true);
 
         runs.put(run.id(), run);
 
         try {
-            ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, cancel);
+            ResultSet rs = executeSqlQueryWithTimer(cacheName, conn, sql, params, true, 0, cancel);
 
             return new KeyValIterator(rs);
         }
@@ -1500,9 +1497,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     @SuppressWarnings("unchecked")
     @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?, ?> cctx, SqlQuery qry) {
         String type = qry.getType();
-        String space = cctx.name();
+        String cacheName = cctx.name();
 
-        TableDescriptor tblDesc = tableDescriptor(type, space);
+        TableDescriptor tblDesc = tableDescriptor(type, cacheName);
 
         if (tblDesc == null)
             throw new IgniteSQLException("Failed to find SQL table for type: " + type,
@@ -1571,10 +1568,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
         GridQueryCancel cancel) {
-        final String space = cctx.name();
+        final String cacheName = cctx.name();
         final String sqlQry = qry.getSql();
 
-        Connection c = connectionForSpace(space);
+        Connection c = connectionForCache(cacheName);
 
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
         final boolean distributedJoins = qry.isDistributedJoins();
@@ -1585,7 +1582,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridCacheTwoStepQuery twoStepQry = null;
         List<GridQueryFieldMetadata> meta;
 
-        final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(space, sqlQry, grpByCollocated,
+        final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(cacheName, sqlQry, grpByCollocated,
             distributedJoins, enforceJoinOrder, qry.isLocal());
         TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
@@ -1680,7 +1677,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                 LinkedHashSet<Integer> caches0 = new LinkedHashSet<>();
 
-                // Setup spaces from schemas.
+                // Setup caches from schemas.
                 assert twoStepQry != null;
 
                 int tblCnt = twoStepQry.tablesCount();
@@ -1689,9 +1686,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     caches0.add(cctx.cacheId());
 
                     for (QueryTable table : twoStepQry.tables()) {
-                        String cacheName = cacheNameForSchemaAndTable(table.schema(), table.table());
+                        String tblCacheName = cacheNameForSchemaAndTable(table.schema(), table.table());
 
-                        int cacheId = CU.cacheId(cacheName);
+                        int cacheId = CU.cacheId(tblCacheName);
 
                         caches0.add(cacheId);
                     }
@@ -1752,8 +1749,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Cache name.
      */
     private String cacheNameForSchemaAndTable(String schemaName, String tblName) {
-        // TODO: This need to be changed.
-        return space(schemaName);
+        return cacheName(schemaName);
     }
 
     /**
@@ -1848,11 +1844,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param type Type description.
      * @throws IgniteCheckedException In case of error.
      */
-    @Override public boolean registerType(String spaceName, GridQueryTypeDescriptor type)
+    @Override public boolean registerType(String cacheName, GridQueryTypeDescriptor type)
         throws IgniteCheckedException {
         validateTypeDescriptor(type);
 
-        String schemaName = schema(spaceName);
+        String schemaName = schema(cacheName);
 
         Schema schema = schemas.get(schemaName);
 
@@ -1861,7 +1857,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             Connection conn = connectionForThread(schemaName);
 
-            createTable(spaceName, schema, tbl, conn);
+            createTable(cacheName, schema, tbl, conn);
 
             schema.add(tbl);
         }
@@ -1969,14 +1965,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Create db table by using given table descriptor.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param schema Schema.
      * @param tbl Table descriptor.
      * @param conn Connection.
      * @throws SQLException If failed to create db table.
      * @throws IgniteCheckedException If failed.
      */
-    private void createTable(String spaceName, Schema schema, TableDescriptor tbl, Connection conn)
+    private void createTable(String cacheName, Schema schema, TableDescriptor tbl, Connection conn)
         throws SQLException, IgniteCheckedException {
         assert schema != null;
         assert tbl != null;
@@ -2011,7 +2007,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridH2Table h2Tbl = H2TableEngine.createTable(conn, sql.toString(), rowDesc, rowFactory, tbl);
 
         for (GridH2IndexBase usrIdx : tbl.createUserIndexes())
-            addInitialUserIndex(spaceName, tbl, usrIdx);
+            addInitialUserIndex(cacheName, tbl, usrIdx);
 
         if (dataTables.putIfAbsent(h2Tbl.identifier(), h2Tbl) != null)
             throw new IllegalStateException("Table already exists: " + h2Tbl.identifierString());
@@ -2069,14 +2065,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Gets table descriptor by type and space names.
+     * Gets table descriptor by type and cache names.
      *
      * @param type Type name.
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @return Table descriptor.
      */
-    @Nullable private TableDescriptor tableDescriptor(String type, String space) {
-        Schema s = schemas.get(schema(space));
+    @Nullable private TableDescriptor tableDescriptor(String type, String cacheName) {
+        Schema s = schemas.get(schema(cacheName));
 
         if (s == null)
             return null;
@@ -2100,13 +2096,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Gets database schema from space.
+     * Gets database schema from cache name.
      *
-     * @param space Space name. {@code null} would be converted to an empty string.
-     * @return Schema name. Should not be null since we should not fail for an invalid space name.
+     * @param cacheName Cache name. {@code null} would be converted to an empty string.
+     * @return Schema name. Should not be null since we should not fail for an invalid cache name.
      */
-    private String schema(String space) {
-        return emptyIfNull(space2schema.get(emptyIfNull(space)));
+    private String schema(String cacheName) {
+        return emptyIfNull(cacheName2schema.get(emptyIfNull(cacheName)));
     }
 
     /**
@@ -2127,7 +2123,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public String space(String schemaName) {
+    @Override public String cacheName(String schemaName) {
         assert schemaName != null;
 
         Schema schema = schemas.get(schemaName);
@@ -2139,19 +2135,19 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             schema = schemas.get(escapeName(schemaName, true));
         }
 
-        return schema.spaceName;
+        return schema.cacheName;
     }
 
     /**
      * Rebuild indexes from hash index.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param type Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    @Override public void rebuildIndexesFromHash(String spaceName,
+    @Override public void rebuildIndexesFromHash(String cacheName,
         GridQueryTypeDescriptor type) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(type.name(), spaceName);
+        TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
 
         if (tbl == null)
             return;
@@ -2208,8 +2204,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void markForRebuildFromHash(String spaceName, GridQueryTypeDescriptor type) {
-        TableDescriptor tbl = tableDescriptor(type.name(), spaceName);
+    @Override public void markForRebuildFromHash(String cacheName, GridQueryTypeDescriptor type) {
+        TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
 
         if (tbl == null)
             return;
@@ -2222,18 +2218,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Gets size (for tests only).
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @return Size.
      * @throws IgniteCheckedException If failed or {@code -1} if the type is unknown.
      */
-    long size(String spaceName, String typeName) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
+    long size(String cacheName, String typeName) throws IgniteCheckedException {
+        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl == null)
             return -1;
 
-        Connection conn = connectionForSpace(spaceName);
+        Connection conn = connectionForCache(cacheName);
 
         setupConnection(conn, false, false);
 
@@ -2519,7 +2515,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         conns.clear();
         schemas.clear();
-        space2schema.clear();
+        cacheName2schema.clear();
 
         try (Connection c = DriverManager.getConnection(dbUrl);
              Statement s = c.createStatement()) {
@@ -2539,14 +2535,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void registerCache(String spaceName, GridCacheContext<?, ?> cctx, CacheConfiguration<?, ?> ccfg)
+    @Override public void registerCache(String cacheName, GridCacheContext<?, ?> cctx, CacheConfiguration<?, ?> ccfg)
         throws IgniteCheckedException {
         String schema = schemaNameFromCacheConf(ccfg);
 
-        if (schemas.putIfAbsent(schema, new Schema(spaceName, schema, cctx, ccfg)) != null)
-            throw new IgniteCheckedException("Cache already registered: " + U.maskName(spaceName));
+        if (schemas.putIfAbsent(schema, new Schema(cacheName, schema, cctx, ccfg)) != null)
+            throw new IgniteCheckedException("Cache already registered: " + U.maskName(cacheName));
 
-        space2schema.put(emptyIfNull(spaceName), schema);
+        cacheName2schema.put(emptyIfNull(cacheName), schema);
 
         createSchema(schema);
 
@@ -2554,14 +2550,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void unregisterCache(String spaceName) {
-        String schema = schema(spaceName);
+    @Override public void unregisterCache(String cacheName) {
+        String schema = schema(cacheName);
         Schema rmv = schemas.remove(schema);
 
         if (rmv != null) {
-            space2schema.remove(emptyIfNull(rmv.spaceName));
-            mapQryExec.onCacheStop(spaceName);
-            dmlProc.onCacheStop(spaceName);
+            cacheName2schema.remove(emptyIfNull(rmv.cacheName));
+            mapQryExec.onCacheStop(cacheName);
+            dmlProc.onCacheStop(cacheName);
 
             rmv.onDrop();
 
@@ -2569,7 +2565,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 dropSchema(schema);
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(spaceName), e);
+                U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(cacheName), e);
             }
 
             for (TableDescriptor tblDesc : rmv.tbls.values())
@@ -2580,7 +2576,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 it.hasNext(); ) {
                 Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery> e = it.next();
 
-                if (F.eq(e.getKey().space, spaceName))
+                if (F.eq(e.getKey().cacheName, cacheName))
                     it.remove();
             }
         }
@@ -2594,8 +2590,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         final AffinityTopologyVersion topVer0 = topVer != null ? topVer : AffinityTopologyVersion.NONE;
 
         return new IndexingQueryFilter() {
-            @Nullable @Override public <K, V> IgniteBiPredicate<K, V> forSpace(String spaceName) {
-                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(spaceName);
+            @Nullable @Override public <K, V> IgniteBiPredicate<K, V> forCache(String cacheName) {
+                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(cacheName);
 
                 if (cache.context().isReplicated())
                     return null;
@@ -2677,7 +2673,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     private static final class TwoStepCachedQueryKey {
         /** */
-        private final String space;
+        private final String cacheName;
 
         /** */
         private final String sql;
@@ -2695,20 +2691,20 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         private final boolean isLocal;
 
         /**
-         * @param space Space.
+         * @param cacheName Cache name.
          * @param sql Sql.
          * @param grpByCollocated Collocated GROUP BY.
          * @param distributedJoins Distributed joins enabled.
          * @param enforceJoinOrder Enforce join order of tables.
          * @param isLocal Query is local flag.
          */
-        private TwoStepCachedQueryKey(String space,
+        private TwoStepCachedQueryKey(String cacheName,
             String sql,
             boolean grpByCollocated,
             boolean distributedJoins,
             boolean enforceJoinOrder,
             boolean isLocal) {
-            this.space = space;
+            this.cacheName = cacheName;
             this.sql = sql;
             this.grpByCollocated = grpByCollocated;
             this.distributedJoins = distributedJoins;
@@ -2735,7 +2731,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             if (enforceJoinOrder != that.enforceJoinOrder)
                 return false;
 
-            if (space != null ? !space.equals(that.space) : that.space != null)
+            if (cacheName != null ? !cacheName.equals(that.cacheName) : that.cacheName != null)
                 return false;
 
             return isLocal == that.isLocal && sql.equals(that.sql);
@@ -2743,7 +2739,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int res = space != null ? space.hashCode() : 0;
+            int res = cacheName != null ? cacheName.hashCode() : 0;
             res = 31 * res + sql.hashCode();
             res = 31 * res + (grpByCollocated ? 1 : 0);
             res = res + (distributedJoins ? 2 : 0);
@@ -3165,7 +3161,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             if (type().valueClass() == String.class) {
                 try {
-                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type);
+                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.cacheName, type);
                 }
                 catch (IgniteCheckedException e1) {
                     throw new IgniteException(e1);
@@ -3178,7 +3174,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             if (textIdx != null) {
                 try {
-                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type);
+                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.cacheName, type);
                 }
                 catch (IgniteCheckedException e1) {
                     throw new IgniteException(e1);
@@ -3450,7 +3446,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     private class Schema {
         /** */
-        private final String spaceName;
+        private final String cacheName;
 
         /** */
         private final String schemaName;
@@ -3471,14 +3467,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         private final CacheConfiguration<?, ?> ccfg;
 
         /**
-         * @param spaceName Space name.
+         * @param cacheName Cache name.
          * @param schemaName Schema name.
          * @param cctx Cache context.
          * @param ccfg Cache configuration.
          */
-        private Schema(String spaceName, String schemaName, GridCacheContext<?, ?> cctx,
+        private Schema(String cacheName, String schemaName, GridCacheContext<?, ?> cctx,
             CacheConfiguration<?, ?> ccfg) {
-            this.spaceName = spaceName;
+            this.cacheName = cacheName;
             this.cctx = cctx;
             this.schemaName = schemaName;
             this.ccfg = ccfg;
@@ -3653,9 +3649,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 CacheObject co = (CacheObject)obj;
 
                 if (type == Value.JAVA_OBJECT)
-                    return new GridH2ValueCacheObject(cacheContext(schema.spaceName), co);
+                    return new GridH2ValueCacheObject(cacheContext(schema.cacheName), co);
 
-                obj = co.value(objectContext(schema.spaceName), false);
+                obj = co.value(objectContext(schema.cacheName), false);
             }
 
             switch (type) {
@@ -3733,7 +3729,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     "or configure key type as common super class for all actual keys for this value type.", e);
             }
 
-            GridCacheContext cctx = cacheContext(schema.spaceName);
+            GridCacheContext cctx = cacheContext(schema.cacheName);
 
             if (cctx.offheapIndex()) {
                 row.ver = ver;
@@ -4013,7 +4009,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         /** {@inheritDoc} */
         @Override public TableBase createTable(CreateTableData createTblData) {
-            resTbl0 = new GridH2Table(createTblData, rowDesc0, rowFactory0, tblDesc0, tblDesc0.schema.spaceName);
+            resTbl0 = new GridH2Table(createTblData, rowDesc0, rowFactory0, tblDesc0, tblDesc0.schema.cacheName);
 
             return resTbl0;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
index 7579fa5..7caf354 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java
@@ -90,9 +90,9 @@ public class H2PkHashIndex extends GridH2IndexBase {
         IgniteBiPredicate<Object, Object> p = null;
 
         if (f != null) {
-            String spaceName = getTable().cacheName();
+            String cacheName = getTable().cacheName();
 
-            p = f.forSpace(spaceName);
+            p = f.forCache(cacheName);
         }
 
         KeyCacheObject lowerObj = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 814df77..c1c1d9c 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
@@ -167,9 +167,9 @@ public class H2TreeIndex extends GridH2IndexBase {
             IgniteBiPredicate<Object, Object> p = null;
 
             if (f != null) {
-                String spaceName = getTable().cacheName();
+                String cacheName = getTable().cacheName();
 
-                p = f.forSpace(spaceName);
+                p = f.forCache(cacheName);
             }
 
             int seg = threadLocalSegment();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index 30fb612..3dabc58 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -1686,18 +1686,16 @@ public abstract class GridH2IndexBase extends BaseIndex {
          * @param cursor GridCursor.
          * @param time Time for expired rows filtering.
          * @param qryFilter Filter.
-         * @param spaceName Space name.
+         * @param cacheName Cache name.
          */
-        protected FilteringCursor(GridCursor<GridH2Row> cursor,
-            long time,
-            IndexingQueryFilter qryFilter,
-            String spaceName) {
+        protected FilteringCursor(GridCursor<GridH2Row> cursor, long time, IndexingQueryFilter qryFilter,
+            String cacheName) {
             this.cursor = cursor;
 
             this.time = time;
 
             if (qryFilter != null) {
-                this.fltr = qryFilter.forSpace(spaceName);
+                this.fltr = qryFilter.forCache(cacheName);
 
                 this.isValRequired = qryFilter.isValueRequired();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 ec728de..cb9042e 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
@@ -222,7 +222,7 @@ public class GridH2Table extends TableBase {
     }
 
     /**
-     * @return Space name.
+     * @return Cache name.
      */
     @Nullable public String cacheName() {
         return cacheName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 39256d4..03fedcb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -206,7 +206,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         int seg = threadLocalSegment();
 
         // Fast path if we don't need to perform any filtering.
-        if (f == null || f.forSpace((getTable()).cacheName()) == null)
+        if (f == null || f.forCache((getTable()).cacheName()) == null)
             try {
                 return treeForRead(seg).size();
             } catch (IgniteCheckedException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
index 6671dc6..48c0cb9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -71,7 +70,7 @@ public class GridLuceneIndex implements AutoCloseable {
     public static final String EXPIRATION_TIME_FIELD_NAME = "_gg_expires__";
 
     /** */
-    private final String spaceName;
+    private final String cacheName;
 
     /** */
     private final GridQueryTypeDescriptor type;
@@ -96,14 +95,14 @@ public class GridLuceneIndex implements AutoCloseable {
      *
      * @param ctx Kernal context.
      * @param mem Unsafe memory.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param type Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
     public GridLuceneIndex(GridKernalContext ctx, @Nullable GridUnsafeMemory mem,
-        @Nullable String spaceName, GridQueryTypeDescriptor type) throws IgniteCheckedException {
+        @Nullable String cacheName, GridQueryTypeDescriptor type) throws IgniteCheckedException {
         this.ctx = ctx;
-        this.spaceName = spaceName;
+        this.cacheName = cacheName;
         this.type = type;
 
         dir = new GridLuceneDirectory(mem == null ? new GridUnsafeMemory(0) : mem);
@@ -141,7 +140,7 @@ public class GridLuceneIndex implements AutoCloseable {
         if (ctx == null)
             return null;
 
-        return ctx.cache().internalCache(spaceName).context().cacheObjectContext();
+        return ctx.cache().internalCache(cacheName).context().cacheObjectContext();
     }
 
     /**
@@ -275,7 +274,7 @@ public class GridLuceneIndex implements AutoCloseable {
         IgniteBiPredicate<K, V> fltr = null;
 
         if (filters != null)
-            fltr = filters.forSpace(spaceName);
+            fltr = filters.forCache(cacheName);
 
         return new It<>(reader, searcher, docs.scoreDocs, fltr);
     }
@@ -382,7 +381,7 @@ public class GridLuceneIndex implements AutoCloseable {
                 ClassLoader ldr = null;
 
                 if (ctx != null && ctx.deploy().enabled())
-                    ldr = ctx.cache().internalCache(spaceName).context().deploy().globalLoader();
+                    ldr = ctx.cache().internalCache(cacheName).context().deploy().globalLoader();
 
                 K k = unmarshall(org.apache.commons.codec.binary.Base64.decodeBase64(doc.get(KEY_FIELD_NAME)), ldr);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 6d76eea..1d8a762 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -579,7 +579,7 @@ public class GridMapQueryExecutor {
                 }
             }
 
-            Connection conn = h2.connectionForSpace(mainCctx.name());
+            Connection conn = h2.connectionForCache(mainCctx.name());
 
             setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 75914ef..1ecbb40 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -540,10 +540,10 @@ public class GridReduceQueryExecutor {
 
             final long qryReqId = qryIdGen.incrementAndGet();
 
-            final String space = cctx.name();
+            final String cacheName = cctx.name();
 
-            final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), space,
-                h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize(),
+            final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), cacheName,
+                h2.connectionForCache(cacheName), qry.mapQueries().size(), qry.pageSize(),
                 U.currentTimeMillis(), cancel);
 
             AffinityTopologyVersion topVer = h2.readyTopologyVersion();
@@ -794,11 +794,11 @@ public class GridReduceQueryExecutor {
 
                         try {
                             if (qry.explain())
-                                return explainPlan(r.conn, space, qry, params);
+                                return explainPlan(r.conn, cacheName, qry, params);
 
                             GridCacheSqlQuery rdc = qry.reduceQuery();
 
-                            ResultSet res = h2.executeSqlQueryWithTimer(space,
+                            ResultSet res = h2.executeSqlQueryWithTimer(cacheName,
                                 r.conn,
                                 rdc.query(),
                                 F.asList(rdc.parameters(params)),
@@ -1030,12 +1030,12 @@ public class GridReduceQueryExecutor {
     }
 
     /**
-     * @param space Cache name.
+     * @param cacheName Cache name.
      * @param topVer Topology version.
      * @return Collection of data nodes.
      */
-    private Collection<ClusterNode> dataNodes(String space, AffinityTopologyVersion topVer) {
-        Collection<ClusterNode> res = ctx.discovery().cacheAffinityNodes(space, topVer);
+    private Collection<ClusterNode> dataNodes(String cacheName, AffinityTopologyVersion topVer) {
+        Collection<ClusterNode> res = ctx.discovery().cacheAffinityNodes(cacheName, topVer);
 
         return res != null ? res : Collections.<ClusterNode>emptySet();
     }
@@ -1049,12 +1049,12 @@ public class GridReduceQueryExecutor {
     private Set<ClusterNode> replicatedUnstableDataNodes(GridCacheContext<?,?> cctx) {
         assert cctx.isReplicated() : cctx.name() + " must be replicated";
 
-        String space = cctx.name();
+        String cacheName = cctx.name();
 
-        Set<ClusterNode> dataNodes = new HashSet<>(dataNodes(space, NONE));
+        Set<ClusterNode> dataNodes = new HashSet<>(dataNodes(cacheName, NONE));
 
         if (dataNodes.isEmpty())
-            throw new CacheException("Failed to find data nodes for cache: " + space);
+            throw new CacheException("Failed to find data nodes for cache: " + cacheName);
 
         // Find all the nodes owning all the partitions for replicated cache.
         for (int p = 0, parts = cctx.affinity().partitions(); p < parts; p++) {
@@ -1213,18 +1213,18 @@ public class GridReduceQueryExecutor {
 
     /**
      * @param c Connection.
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param params Query parameters.
      * @return Cursor for plans.
      * @throws IgniteCheckedException if failed.
      */
-    private Iterator<List<?>> explainPlan(JdbcConnection c, String space, GridCacheTwoStepQuery qry, Object[] params)
+    private Iterator<List<?>> explainPlan(JdbcConnection c, String cacheName, GridCacheTwoStepQuery qry, Object[] params)
         throws IgniteCheckedException {
         List<List<?>> lists = new ArrayList<>();
 
         for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) {
-            ResultSet rs = h2.executeSqlQueryWithTimer(space, c,
+            ResultSet rs = h2.executeSqlQueryWithTimer(cacheName, c,
                 "SELECT PLAN FROM " + mergeTableIdentifier(i), null, false, 0, null);
 
             lists.add(F.asList(getPlan(rs)));
@@ -1240,7 +1240,7 @@ public class GridReduceQueryExecutor {
 
         GridCacheSqlQuery rdc = qry.reduceQuery();
 
-        ResultSet rs = h2.executeSqlQueryWithTimer(space,
+        ResultSet rs = h2.executeSqlQueryWithTimer(cacheName,
             c,
             "EXPLAIN " + rdc.query(),
             F.asList(rdc.parameters(params)),

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
index 1241d0c..7e3473f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
@@ -1020,13 +1020,13 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
     /**
      * Synchronously create index.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @param idx Index.
      * @param ifNotExists When set to true operation will fail if index already exists.
      * @throws Exception If failed.
      */
-    private void dynamicIndexCreate(String space, String tblName, QueryIndex idx, boolean ifNotExists)
+    private void dynamicIndexCreate(String cacheName, String tblName, QueryIndex idx, boolean ifNotExists)
         throws Exception {
         GridStringBuilder sql = new SB("CREATE INDEX ")
             .a(ifNotExists ? "IF NOT EXISTS " : "")
@@ -1051,32 +1051,32 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
 
         sql.a(')');
 
-        executeSql(space, sql.toString());
+        executeSql(cacheName, sql.toString());
     }
 
     /**
      * Synchronously drop index.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param idxName Index name.
      * @param ifExists When set to true operation fill fail if index doesn't exists.
      * @throws Exception if failed.
      */
-    private void dynamicIndexDrop(String space, String idxName, boolean ifExists) throws Exception {
+    private void dynamicIndexDrop(String cacheName, String idxName, boolean ifExists) throws Exception {
         String sql = "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + "\"" + idxName + "\"";
 
-        executeSql(space, sql);
+        executeSql(cacheName, sql);
     }
 
     /**
      * Execute SQL.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param sql SQL.
      */
-    private void executeSql(String space, String sql) {
+    private void executeSql(String cacheName, String sql) {
         log.info("Executing DDL: " + sql);
 
-        node().cache(space).query(new SqlFieldsQuery(sql)).getAll();
+        node().cache(cacheName).query(new SqlFieldsQuery(sql)).getAll();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
index f9e2f75..3600022 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
@@ -1033,20 +1033,20 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     private static class BlockingIndexing extends IgniteH2Indexing {
         /** {@inheritDoc} */
-        @Override public void dynamicIndexCreate(@NotNull String spaceName, String tblName,
+        @Override public void dynamicIndexCreate(@NotNull String cacheName, String tblName,
             QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor)
             throws IgniteCheckedException {
             awaitIndexing(ctx.localNodeId());
 
-            super.dynamicIndexCreate(spaceName, tblName, idxDesc, ifNotExists, cacheVisitor);
+            super.dynamicIndexCreate(cacheName, tblName, idxDesc, ifNotExists, cacheVisitor);
         }
 
         /** {@inheritDoc} */
-        @Override public void dynamicIndexDrop(@NotNull String spaceName, String idxName, boolean ifExists)
+        @Override public void dynamicIndexDrop(@NotNull String cacheName, String idxName, boolean ifExists)
             throws IgniteCheckedException{
             awaitIndexing(ctx.localNodeId());
 
-            super.dynamicIndexDrop(spaceName, idxName, ifExists);
+            super.dynamicIndexDrop(cacheName, idxName, ifExists);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java
index ddaea8a..13c0cb2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java
@@ -191,7 +191,7 @@ public class IgniteQueryDedicatedPoolTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) {
             return idx.containsKey(GridIoPolicy.QUERY_POOL) ?
                 Collections.<Cache.Entry<?, ?>>singletonList(
@@ -200,12 +200,12 @@ public class IgniteQueryDedicatedPoolTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime) {
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime) {
             idx.put(key, val);
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) {
+        @Override public void remove(@Nullable String cacheName, Object key) {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 3832878..7ba7d56 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -246,32 +246,32 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
     public void testSpi() throws Exception {
         IgniteH2Indexing spi = getIndexing();
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(-1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(-1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(-1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(-1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         IgniteCache<Integer, BinaryObject> cacheA = ignite0.createCache(cacheACfg());
 
-        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(0, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(-1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(0, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(-1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         IgniteCache<Integer, BinaryObject> cacheB = ignite0.createCache(cacheBCfg());
 
         // Initially all is empty.
-        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(0, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(0, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name()));
 
-        assertFalse(spi.queryLocalSql(typeAA.space(), "select * from A.A", null, Collections.emptySet(), typeAA.name(), null, null).hasNext());
-        assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", null, Collections.emptySet(), typeAB.name(), null, null).hasNext());
-        assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(typeAA.cacheName(), "select * from A.A", null, Collections.emptySet(), typeAA.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(typeAB.cacheName(), "select * from A.B", null, Collections.emptySet(), typeAB.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(typeBA.cacheName(), "select * from B.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext());
 
-        assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A, A.B, A.A", null,
+        assertFalse(spi.queryLocalSql(typeBA.cacheName(), "select * from B.A, A.B, A.A", null,
             Collections.emptySet(), typeBA.name(), null, null).hasNext());
 
         try {
-            spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null,
+            spi.queryLocalSql(typeBA.cacheName(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null,
                 Collections.emptySet(), typeBA.name(), null, null).hasNext();
 
             fail("Enumerations of aliases in select block must be prohibited");
@@ -280,60 +280,60 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             // all fine
         }
 
-        assertFalse(spi.queryLocalSql(typeAB.space(), "select ab.* from A.B ab", null,
+        assertFalse(spi.queryLocalSql(typeAB.cacheName(), "select ab.* from A.B ab", null,
             Collections.emptySet(), typeAB.name(), null, null).hasNext());
 
-        assertFalse(spi.queryLocalSql(typeBA.space(), "select   ba.*   from B.A  as ba", null,
+        assertFalse(spi.queryLocalSql(typeBA.cacheName(), "select   ba.*   from B.A  as ba", null,
             Collections.emptySet(), typeBA.name(), null, null).hasNext());
 
         cacheA.put(1, aa("A", 1, "Vasya", 10).build());
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(0, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(0, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheA.put(1, ab(1, "Vasya", 20, "Some text about Vasya goes here.").build());
 
-        // In one space all keys must be unique.
-        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
+        // In one cache all keys must be unique.
+        assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheB.put(1, ba(2, "Petya", 25, true).build());
 
-        // No replacement because of different space.
-        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
+        // No replacement because of different cache.
+        assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheB.put(1, ba(2, "Kolya", 25, true).build());
 
         // Replacement in the same table.
-        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheA.put(2, aa("A", 2, "Valera", 19).build());
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheA.put(3, aa("A", 3, "Borya", 18).build());
 
-        assertEquals(2, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(2, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheA.put(4, ab(4, "Vitalya", 20, "Very Good guy").build());
 
-        assertEquals(2, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(2, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         // Query data.
         Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> res =
-            spi.queryLocalSql(typeAA.space(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null);
+            spi.queryLocalSql(typeAA.cacheName(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next()));
@@ -341,7 +341,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age", null,
+        res = spi.queryLocalSql(typeAA.cacheName(), "select aa.* from a aa order by aa.age", null,
             Collections.emptySet(), typeAA.name(), null, null);
 
         assertTrue(res.hasNext());
@@ -350,7 +350,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(typeAB.space(), "from b order by name", null, Collections.emptySet(), typeAB.name(), null, null);
+        res = spi.queryLocalSql(typeAB.cacheName(), "from b order by name", null, Collections.emptySet(), typeAB.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").build(), value(res.next()));
@@ -358,7 +358,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name", null,
+        res = spi.queryLocalSql(typeAB.cacheName(), "select bb.* from b as bb order by bb.name", null,
             Collections.emptySet(), typeAB.name(), null, null);
 
         assertTrue(res.hasNext());
@@ -368,14 +368,14 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertFalse(res.hasNext());
 
 
-        res = spi.queryLocalSql(typeBA.space(), "from a", null, Collections.emptySet(), typeBA.name(), null, null);
+        res = spi.queryLocalSql(typeBA.cacheName(), "from a", null, Collections.emptySet(), typeBA.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(ba(2, "Kolya", 25, true).build(), value(res.next()));
         assertFalse(res.hasNext());
 
         // Text queries
-        Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> txtRes = spi.queryLocalText(typeAB.space(), "good",
+        Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> txtRes = spi.queryLocalText(typeAB.cacheName(), "good",
             typeAB.name(), null);
 
         assertTrue(txtRes.hasNext());
@@ -410,32 +410,32 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         // Remove
         cacheA.remove(2);
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name()));
 
         cacheB.remove(1);
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name()));
 
         // Unregister.
-        spi.unregisterType(typeAA.space(), typeAA.name());
+        spi.unregisterType(typeAA.cacheName(), typeAA.name());
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name()));
 
-        spi.unregisterType(typeAB.space(), typeAB.name());
+        spi.unregisterType(typeAB.cacheName(), typeAB.name());
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
-        assertEquals(-1, spi.size(typeAB.space(), typeAB.name()));
-        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
+        assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name()));
+        assertEquals(-1, spi.size(typeAB.cacheName(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name()));
 
-        spi.unregisterType(typeBA.space(), typeBA.name());
+        spi.unregisterType(typeBA.cacheName(), typeBA.name());
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name()));
     }
 
     /**
@@ -533,7 +533,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         private final String name;
 
         /** */
-        private final String space;
+        private final String cacheName;
 
         /** */
         private final Map<String, Class<?>> valFields;
@@ -542,14 +542,14 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         private final GridQueryIndexDescriptor textIdx;
 
         /**
-         * @param space Space name.
+         * @param cacheName Cache name.
          * @param name Type name.
          * @param valFields Fields.
          * @param textIdx Fulltext index.
          */
-        private TypeDesc(String space, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
+        private TypeDesc(String cacheName, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
             this.name = name;
-            this.space = space;
+            this.cacheName = cacheName;
             this.valFields = Collections.unmodifiableMap(valFields);
             this.textIdx = textIdx;
         }
@@ -570,10 +570,10 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /**
-         * @return Space name.
+         * @return Cache name.
          */
-        String space() {
-            return space;
+        String cacheName() {
+            return cacheName;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index 21666d6..9838084 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -686,7 +686,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
 
         IgniteH2Indexing idx = U.field(qryProcessor, "idx");
 
-        return (JdbcConnection)idx.connectionForSpace(DEFAULT_CACHE_NAME);
+        return (JdbcConnection)idx.connectionForCache(DEFAULT_CACHE_NAME);
     }
 
     /**


[19/39] ignite git commit: Master: added cache atomicity log on cache start (minor).

Posted by sb...@apache.org.
Master: added cache atomicity log on cache start (minor).


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: ffb4f3c2f17b308ce841c4798c5eb5dcf49602d7
Parents: 8c75e4d
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 25 10:14:21 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 25 10:14:21 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ffb4f3c2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 06ad62d..a716ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1165,7 +1165,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheCtx.onStarted();
 
         if (log.isInfoEnabled())
-            log.info("Started cache [name=" + U.maskName(cfg.getName()) + ", memoryPolicyName=" + cfg.getMemoryPolicyName() + ", mode=" + cfg.getCacheMode() + ']');
+            log.info("Started cache [name=" + U.maskName(cfg.getName()) +
+                ", memoryPolicyName=" + cfg.getMemoryPolicyName() + ", mode=" + cfg.getCacheMode() +
+                ", atomicity=" + cfg.getAtomicityMode() + ']');
     }
 
     /**


[33/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: e24b538d106aa219d6bd141a699eaf879753cc80
Parents: 674e7dd
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 12:46:16 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 12:46:16 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryEventBuffer.java        | 44 +++++++-------------
 1 file changed, 15 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e24b538d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index afe34c4..acc6d50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -81,40 +81,26 @@ public class CacheContinuousQueryEventBuffer {
     @Nullable Collection<CacheContinuousQueryEntry> flushOnExchange() {
         Collection<CacheContinuousQueryEntry> ret = null;
 
-        for (;;) {
-            Batch batch = curBatch.get();
-
-            if (batch != null) {
-                Collection<CacheContinuousQueryEntry> ret0 = batch.flushAndReset();
-
-                if (ret0 != null) {
-                    if (ret == null)
-                        ret = ret0;
-                    else
-                        ret.addAll(ret0);
-                }
-            }
+        Batch batch = curBatch.get();
 
-            if (!backupQ.isEmpty()) {
-                if (ret == null)
-                    ret = new ArrayList<>();
+        if (batch != null)
+            ret = batch.flushCurrentEntries();
 
-                CacheContinuousQueryEntry e;
+        if (!backupQ.isEmpty()) {
+            if (ret == null)
+                ret = new ArrayList<>();
 
-                while ((e = backupQ.pollFirst()) != null)
-                    ret.add(e);
-            }
+            CacheContinuousQueryEntry e;
 
-            if (!pending.isEmpty()) {
-                if (ret == null)
-                    ret = new ArrayList<>();
+            while ((e = backupQ.pollFirst()) != null)
+                ret.add(e);
+        }
 
-                ret.addAll(pending.values());
-            }
+        if (!pending.isEmpty()) {
+            if (ret == null)
+                ret = new ArrayList<>();
 
-            break;
-//            if (curBatch.compareAndSet(batch, null))
-//                break;
+            ret.addAll(pending.values());
         }
 
         return ret;
@@ -330,7 +316,7 @@ public class CacheContinuousQueryEventBuffer {
         /**
          * @return Entries to send as part of backup queue.
          */
-        @Nullable synchronized List<CacheContinuousQueryEntry> flushAndReset() {
+        @Nullable synchronized List<CacheContinuousQueryEntry> flushCurrentEntries() {
             if (entries == null)
                 return null;
 


[14/39] ignite git commit: IGNITE-5121: REST API call with empty cache name to call "default." - Fixes #1920.

Posted by sb...@apache.org.
IGNITE-5121: REST API call with empty cache name to call "default." - Fixes #1920.

Signed-off-by: shroman <rs...@yahoo.com>


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: d8eeea84bef33d33470052fbb86c96d75490c9b8
Parents: 8c9b1bd
Author: shroman <rs...@yahoo.com>
Authored: Wed May 24 17:19:26 2017 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Wed May 24 17:19:26 2017 +0900

----------------------------------------------------------------------
 .../rest/handlers/GridRestCommandHandlerAdapter.java   |  3 +++
 .../rest/handlers/cache/GridCacheCommandHandler.java   |  2 +-
 .../rest/handlers/query/QueryCommandHandler.java       | 13 +++++++------
 3 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d8eeea84/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandlerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandlerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandlerAdapter.java
index 31c26b3..def11dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandlerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandlerAdapter.java
@@ -24,6 +24,9 @@ import org.apache.ignite.internal.GridKernalContext;
  * Abstract command handler.
  */
 public abstract class GridRestCommandHandlerAdapter implements GridRestCommandHandler {
+    /** Used cache name in case the name was not defined in a request. */
+    protected static final String DFLT_CACHE_NAME = "default";
+
     /** Kernal context. */
     protected final GridKernalContext ctx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d8eeea84/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 18cd6af..9edf37e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -360,7 +360,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         GridRestCacheRequest req0 = (GridRestCacheRequest)req;
 
-        final String cacheName = req0.cacheName();
+        final String cacheName = req0.cacheName() == null ? DFLT_CACHE_NAME: req0.cacheName();
 
         final Object key = req0.key();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d8eeea84/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
index ee728a6..e7cc3dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
@@ -219,11 +219,11 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         assert req instanceof RestQueryRequest : "Invalid type of query request.";
 
         if (req.command() != CLOSE_SQL_QUERY) {
-            Integer pageSize = ((RestQueryRequest) req).pageSize();
+            Integer pageSize = ((RestQueryRequest)req).pageSize();
 
             if (pageSize == null)
                 return new GridFinishedFuture<>(
-                        new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("pageSize"))
+                    new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("pageSize"))
                 );
         }
 
@@ -232,17 +232,17 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
             case EXECUTE_SQL_FIELDS_QUERY:
             case EXECUTE_SCAN_QUERY: {
                 return ctx.closure().callLocalSafe(
-                        new ExecuteQueryCallable(ctx, (RestQueryRequest) req, qryCurs), false);
+                    new ExecuteQueryCallable(ctx, (RestQueryRequest)req, qryCurs), false);
             }
 
             case FETCH_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                        new FetchQueryCallable((RestQueryRequest) req, qryCurs), false);
+                    new FetchQueryCallable((RestQueryRequest)req, qryCurs), false);
             }
 
             case CLOSE_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                        new CloseQueryCallable((RestQueryRequest) req, qryCurs), false);
+                    new CloseQueryCallable((RestQueryRequest)req, qryCurs), false);
             }
         }
 
@@ -315,7 +315,8 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
                         throw new IgniteException("Incorrect query type [type=" + req.queryType() + "]");
                 }
 
-                IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName());
+                IgniteCache<Object, Object> cache = ctx.grid().cache(
+                    req.cacheName() == null ? DFLT_CACHE_NAME : req.cacheName());
 
                 if (cache == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,


[08/39] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite

Posted by sb...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: eea7a3b914f59df1bb4e2b2d27ca36af2acdde2f
Parents: b887587 cbf0b2a
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue May 23 13:52:56 2017 -0700
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue May 23 13:52:56 2017 -0700

----------------------------------------------------------------------
 examples/config/filesystem/example-igfs.xml     |    4 +
 .../store/cassandra/CassandraCacheStore.java    |   15 +-
 .../ignite/tests/IgnitePersistentStoreTest.java |   62 +-
 .../persistence/loadall_blob/ignite-config.xml  |   90 ++
 .../loadall_blob/persistence-settings.xml       |   29 +
 modules/clients/src/test/config/jdbc-config.xml |    6 -
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |    5 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |   24 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |    4 +-
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |  121 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    3 +
 .../jdbc/thin/JdbcConnectionSelfTest.java       |  195 +++
 .../java/org/apache/ignite/IgniteCache.java     |   13 +
 .../org/apache/ignite/IgniteJdbcThinDriver.java |  312 ++++
 .../apache/ignite/IgniteSystemProperties.java   |    7 +-
 .../ignite/binary/BinaryBasicNameMapper.java    |    2 +-
 .../rendezvous/RendezvousAffinityFunction.java  |   20 +-
 .../ignite/cache/query/FieldsQueryCursor.java   |   39 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |    6 +-
 .../configuration/AtomicConfiguration.java      |   25 +
 .../configuration/MemoryConfiguration.java      |    3 +-
 .../MemoryPolicyConfiguration.java              |    4 +-
 .../ignite/configuration/TopologyValidator.java |   10 +-
 .../internal/ComputeTaskInternalFuture.java     |    3 -
 .../ignite/internal/GridKernalContext.java      |    8 +-
 .../ignite/internal/GridKernalContextImpl.java  |   12 +-
 .../apache/ignite/internal/IgniteKernal.java    |    6 +-
 .../internal/IgniteNeedReconnectException.java  |    2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    1 +
 .../internal/binary/BinaryClassDescriptor.java  |    1 +
 .../ignite/internal/binary/BinaryContext.java   |   58 +-
 .../internal/binary/BinaryWriterExImpl.java     |   16 +-
 .../GridClientOptimizedMarshaller.java          |    2 +-
 .../GridClientZipOptimizedMarshaller.java       |    2 +-
 .../internal/jdbc/thin/JdbcConnection.java      |  529 +++++++
 .../ignite/internal/jdbc/thin/JdbcTcpIo.java    |  207 +++
 .../ignite/internal/jdbc2/JdbcConnection.java   |    3 +-
 .../managers/communication/GridIoManager.java   |  366 ++++-
 .../communication/GridIoMessageFactory.java     |    2 +
 .../communication/IgniteIoTestMessage.java      |  362 ++++-
 .../discovery/GridDiscoveryManager.java         |   19 +-
 .../managers/indexing/GridIndexingManager.java  |   19 +-
 .../optimized/OptimizedClassDescriptor.java     |    2 +-
 .../optimized/OptimizedMarshaller.java          |    2 +-
 .../marshaller/optimized/package-info.java      |   22 +
 .../mem/unsafe/UnsafeMemoryProvider.java        |   18 +-
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |   22 +-
 .../affinity/AffinityTopologyVersion.java       |    9 +
 .../affinity/GridAffinityAssignmentCache.java   |    3 -
 .../cache/CacheAffinitySharedManager.java       |  286 ++--
 .../CacheClientReconnectDiscoveryData.java      |  133 ++
 .../internal/processors/cache/CacheData.java    |  157 ++
 .../cache/CacheJoinNodeDiscoveryData.java       |  147 ++
 .../cache/CacheNodeCommonDiscoveryData.java     |   82 +
 .../cache/CacheStoreBalancingWrapper.java       |    3 -
 .../processors/cache/ClusterCachesInfo.java     |  913 +++++++++++
 .../cache/DynamicCacheChangeBatch.java          |   92 +-
 .../cache/DynamicCacheChangeRequest.java        |  102 +-
 .../cache/DynamicCacheDescriptor.java           |  171 +--
 .../processors/cache/ExchangeActions.java       |  338 +++++
 .../processors/cache/GridCacheAdapter.java      |   12 +-
 .../cache/GridCacheAffinityManager.java         |    2 +-
 .../processors/cache/GridCacheContext.java      |   48 +-
 .../cache/GridCacheDeploymentManager.java       |    2 +-
 .../cache/GridCacheFutureAdapter.java           |    1 +
 .../processors/cache/GridCacheLogger.java       |    2 +-
 .../cache/GridCacheMultiTxFuture.java           |    3 -
 .../processors/cache/GridCacheMvccManager.java  |    3 -
 .../GridCachePartitionExchangeManager.java      |   57 +-
 .../processors/cache/GridCacheProcessor.java    | 1413 +++++-------------
 .../cache/GridCacheSharedContext.java           |   17 +-
 .../processors/cache/GridCacheUtils.java        |  146 +-
 .../processors/cache/IgniteCacheProxy.java      |   11 +-
 .../processors/cache/QueryCursorImpl.java       |   19 +-
 .../IgniteCacheDatabaseSharedManager.java       |   51 +-
 .../dht/GridClientPartitionTopology.java        |    2 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |   36 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |   36 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |   63 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    3 -
 .../distributed/dht/GridDhtEmbeddedFuture.java  |    3 -
 .../distributed/dht/GridDhtGetSingleFuture.java |    3 -
 .../dht/GridDhtPartitionTopologyImpl.java       |   12 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    3 -
 .../GridDhtAtomicAbstractUpdateRequest.java     |    2 +
 .../dht/preloader/GridDhtForceKeysFuture.java   |    7 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   13 +-
 .../GridDhtPartitionsExchangeFuture.java        |  166 +-
 .../dht/preloader/GridDhtPreloader.java         |    4 +-
 .../distributed/near/GridNearGetFuture.java     |    3 -
 .../distributed/near/GridNearLockFuture.java    |    3 -
 .../processors/cache/local/GridLocalCache.java  |   31 +-
 .../cache/local/GridLocalLockFuture.java        |   41 +-
 .../cache/query/GridCacheQueryManager.java      |   53 +-
 .../continuous/CacheContinuousQueryHandler.java |   42 +-
 .../continuous/CacheContinuousQueryManager.java |   11 +-
 .../store/GridCacheStoreManagerAdapter.java     |    4 +-
 .../cluster/GridClusterStateProcessor.java      |   26 +-
 .../continuous/GridContinuousProcessor.java     |    3 -
 .../datastreamer/DataStreamerFuture.java        |    3 -
 .../GridAtomicCacheQueueImpl.java               |    7 +-
 .../processors/igfs/IgfsDataManager.java        |    3 -
 .../marshaller/MarshallerMappingTransport.java  |    3 -
 .../processors/odbc/OdbcNioListener.java        |  242 ---
 .../internal/processors/odbc/OdbcProcessor.java |  199 ---
 .../odbc/SqlListenerAbstractMessageParser.java  |  265 ++++
 .../odbc/SqlListenerAbstractObjectReader.java   |  137 ++
 .../odbc/SqlListenerAbstractObjectWriter.java   |  111 ++
 .../processors/odbc/SqlListenerNioListener.java |  263 ++++
 .../processors/odbc/SqlListenerProcessor.java   |  191 +++
 .../odbc/SqlListenerRequestHandlerImpl.java     |  494 ++++++
 .../processors/odbc/jdbc/JdbcMessageParser.java |   50 +
 .../processors/odbc/jdbc/JdbcObjectReader.java  |   33 +
 .../processors/odbc/jdbc/JdbcObjectWriter.java  |   33 +
 .../processors/odbc/odbc/OdbcMessageParser.java |  249 +--
 .../processors/odbc/odbc/OdbcObjectReader.java  |   33 +
 .../processors/odbc/odbc/OdbcObjectWriter.java  |   32 +
 .../odbc/odbc/OdbcRequestHandler.java           |  513 -------
 .../platform/cache/PlatformCache.java           |   28 +-
 .../processors/query/GridQueryIndexing.java     |   90 +-
 .../processors/query/GridQueryProcessor.java    |  289 ++--
 .../processors/query/QueryIndexKey.java         |   20 +-
 .../query/QueryTypeDescriptorImpl.java          |   16 +-
 .../processors/query/QueryTypeIdKey.java        |   26 +-
 .../processors/query/QueryTypeNameKey.java      |   12 +-
 .../internal/processors/query/QueryUtils.java   |   14 +-
 .../schema/SchemaIndexCacheVisitorImpl.java     |   12 +-
 .../processors/query/schema/SchemaKey.java      |   59 -
 .../message/SchemaProposeDiscoveryMessage.java  |    7 +-
 .../operation/SchemaAbstractOperation.java      |   27 +-
 .../operation/SchemaIndexAbstractOperation.java |    7 +-
 .../operation/SchemaIndexCreateOperation.java   |    8 +-
 .../operation/SchemaIndexDropOperation.java     |    7 +-
 .../service/GridServiceDeploymentFuture.java    |    3 -
 .../ignite/internal/util/IgniteUtils.java       |   17 +
 .../util/future/GridEmbeddedFuture.java         |    3 -
 .../util/nio/GridNioEmbeddedFuture.java         |    3 -
 .../internal/util/nio/GridNioFutureImpl.java    |    3 -
 .../ignite/internal/util/nio/GridNioServer.java |    3 -
 .../internal/util/worker/GridWorkerFuture.java  |    3 -
 .../node/VisorAffinityTopologyVersion.java      |   87 ++
 .../visor/node/VisorNodeDataCollectorJob.java   |    6 +
 .../node/VisorNodeDataCollectorJobResult.java   |   38 +
 .../visor/node/VisorNodeDataCollectorTask.java  |    4 +
 .../node/VisorNodeDataCollectorTaskResult.java  |   28 +-
 .../apache/ignite/marshaller/Marshaller.java    |    3 -
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    3 -
 .../ignite/spi/discovery/DiscoveryDataBag.java  |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    4 +-
 .../spi/indexing/IndexingQueryFilter.java       |    8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   17 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |    6 +-
 modules/core/src/test/config/example-cache.xml  |    5 +-
 .../core/src/test/config/examples.properties    |    1 +
 .../src/test/config/spring-start-nodes-attr.xml |    4 +-
 .../core/src/test/config/spring-start-nodes.xml |    4 +-
 .../config/websession/example-cache-base.xml    |    5 +-
 .../IgniteExternalizableAbstractTest.java       |   15 +-
 ...ityFunctionBackupFilterAbstractSelfTest.java |   13 +-
 ...dbcPojoStoreOptimizedMarshallerSelfTest.java |   31 -
 ...ptimizedMarshallerWithSqlEscapeSelfTest.java |   28 -
 .../apache/ignite/igfs/IgfsPathSelfTest.java    |    7 +-
 .../internal/GridLifecycleAwareSelfTest.java    |    3 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |   13 +-
 .../BinarySimpleNameTestPropertySelfTest.java   |   17 +
 .../managers/GridManagerStopSelfTest.java       |    3 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |    2 +-
 .../optimized/OptimizedMarshallerTest.java      |    8 +
 .../marshaller/optimized/package-info.java      |   22 +
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |   31 +-
 .../AtomicCacheAffinityConfigurationTest.java   |  175 +++
 .../processors/cache/CacheGetFromJobTest.java   |    5 +
 .../CacheStartupInDeploymentModesTest.java      |   37 -
 .../GridCacheAbstractLocalStoreSelfTest.java    |   41 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |    2 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |   12 -
 ...ridCacheStoreManagerDeserializationTest.java |    4 +-
 .../cache/GridCacheVersionSelfTest.java         |    7 +-
 .../cache/IgniteCacheContainsKeyAtomicTest.java |  103 ++
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |    6 +-
 .../processors/cache/IgniteCacheStartTest.java  |  191 +++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |    2 +-
 ...niteTopologyValidatorGridSplitCacheTest.java |    8 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |    6 +-
 .../distributed/CacheAffinityEarlyTest.java     |    7 +-
 .../CacheDiscoveryDataConcurrentJoinTest.java   |  198 +++
 .../CacheGetFutureHangsSelfTest.java            |    6 +-
 .../CacheLateAffinityAssignmentTest.java        |    4 +-
 .../cache/distributed/CacheStartOnJoinTest.java |   10 +
 .../IgniteCacheCreatePutMultiNodeSelfTest.java  |    7 +-
 .../distributed/IgniteCacheCreatePutTest.java   |    7 +-
 .../IgniteCachePartitionLossPolicySelfTest.java |    1 +
 .../IgniteCrossCacheTxStoreSelfTest.java        |   12 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |   22 +-
 .../paged/PageEvictionMultinodeTest.java        |   22 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |   14 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |    6 +-
 .../CacheEntryProcessorNonSerializableTest.java |    5 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          |   10 -
 .../processors/igfs/IgfsFileInfoSelfTest.java   |   16 +-
 .../IgfsPrimaryOptimziedMarshallerSelfTest.java |   28 -
 .../odbc/OdbcProcessorValidationSelfTest.java   |  182 ---
 .../SqlListenerProcessorValidationSelfTest.java |  184 +++
 .../service/ClosureServiceClientsNodesTest.java |    4 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +
 ...namicProxySerializationMultiJvmSelfTest.java |   13 -
 .../GridMarshallerPerformanceTest.java          |    8 +-
 .../IgniteMessagingWithClientTest.java          |    4 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |    4 +-
 .../TcpDiscoveryMarshallerCheckSelfTest.java    |    4 +-
 .../ignite/testframework/GridTestUtils.java     |    2 +-
 .../config/GridTestProperties.java              |    3 +-
 .../configvariations/ConfigVariations.java      |   14 +-
 .../testframework/junits/GridAbstractTest.java  |   33 +
 .../junits/IgniteTestResources.java             |    8 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    8 +
 ...naryObjectsTxDeadlockDetectionTestSuite.java |   37 -
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   14 +-
 .../IgniteBinaryCacheFullApiTestSuite.java      |   37 -
 .../testsuites/IgniteBinaryCacheTestSuite.java  |    4 -
 ...ObjectsCacheDataStructuresSelfTestSuite.java |   42 -
 ...BinaryObjectsCacheExpiryPolicyTestSuite.java |   38 -
 ...gniteBinaryObjectsCacheRestartTestSuite.java |   37 -
 .../IgniteBinaryObjectsCacheTestSuite2.java     |   37 -
 .../IgniteBinaryObjectsCacheTestSuite3.java     |    2 -
 .../IgniteBinaryObjectsCacheTestSuite4.java     |   37 -
 ...IgniteBinaryObjectsComputeGridTestSuite.java |    4 -
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 -
 .../testsuites/IgniteCacheTestSuite3.java       |    2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    7 +
 ...teBasicConfigVariationsFullApiTestSuite.java |    2 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    3 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |    4 -
 .../IgniteMarshallerSelfTestSuite.java          |    8 +-
 ...essagingConfigVariationFullApiTestSuite.java |    2 +-
 ...ServiceConfigVariationsFullApiTestSuite.java |    2 +-
 .../IgniteTopologyValidatorTestSuit.java        |   44 -
 .../IgniteTopologyValidatorTestSuite.java       |   50 +
 .../webapp/META-INF/ignite-webapp-config.xml    |   15 +-
 .../hadoop/impl/igfs/HadoopIgfsFuture.java      |    3 -
 .../external/HadoopExternalTaskExecutor.java    |    3 -
 .../test/config/hadoop-fs-open-test/grid-0.xml  |    3 +-
 .../test/config/hadoop-fs-open-test/grid-1.xml  |    3 +-
 .../test/config/hadoop-fs-open-test/grid-2.xml  |    3 +-
 .../test/config/igfs-cli-config-dual-async.xml  |    3 +-
 .../test/config/igfs-cli-config-dual-sync.xml   |    3 +-
 .../src/test/config/igfs-cli-config-primary.xml |    3 +-
 .../src/test/config/igfs-cli-config-proxy.xml   |    3 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |    4 +-
 .../cache/query/GridCacheTwoStepQuery.java      |   84 +-
 .../processors/cache/query/QueryTable.java      |  164 ++
 .../query/h2/DmlStatementsProcessor.java        |   53 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  368 ++---
 .../query/h2/database/H2PkHashIndex.java        |    4 +-
 .../query/h2/database/H2TreeIndex.java          |    4 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |   25 +-
 .../query/h2/opt/GridH2CollocationModel.java    |    6 +-
 .../query/h2/opt/GridH2IndexBase.java           |   14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  177 ++-
 .../query/h2/opt/GridH2TreeIndex.java           |    2 +-
 .../query/h2/opt/GridLuceneIndex.java           |   15 +-
 .../query/h2/sql/GridSqlDropIndex.java          |   14 +-
 .../query/h2/sql/GridSqlQueryParser.java        |    2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   21 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   21 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  263 ++--
 .../h2/twostep/msg/GridH2QueryRequest.java      |   13 +-
 .../twostep/msg/GridH2ValueMessageFactory.java  |    4 +
 .../cache/IgniteCacheNoClassQuerySelfTest.java  |    4 +-
 .../cache/SqlFieldsQuerySelfTest.java           |   12 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |   26 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |   74 +-
 .../cache/index/SchemaExchangeSelfTest.java     |    2 +-
 .../query/IgniteQueryDedicatedPoolTest.java     |    6 +-
 .../query/IgniteSqlSegmentedIndexSelfTest.java  |   25 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  136 +-
 .../h2/sql/AbstractH2CompareQueryTest.java      |    4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |    6 +-
 .../FetchingQueryCursorStressTest.java          |    4 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |    4 -
 .../IgniteBinaryCacheQueryTestSuite2.java       |    4 -
 .../IgniteBinaryCacheQueryTestSuite3.java       |   39 -
 .../IgniteBinaryCacheQueryTestSuite4.java       |   39 -
 .../ignite/ml/math/d3-matrix-template.html      |   17 +-
 .../ignite/ml/math/d3-vector-template.html      |   15 +-
 .../core/include/ignite/cache/query/query_sql.h |    2 +
 .../ignite/cache/query/query_sql_fields.h       |    3 +
 .../cpp/odbc/include/ignite/odbc/message.h      |   10 +
 .../Apache.Ignite.Core.Tests.csproj             |    1 +
 .../Binary/BinaryDateTimeTest.cs                |  204 +++
 .../Binary/BinarySelfTest.cs                    |    2 +-
 .../Cache/Query/CacheLinqTest.cs                |   38 +-
 .../Cache/Query/CacheQueriesTest.cs             |   60 +-
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |    2 +-
 .../Apache.Ignite.Core.csproj                   |    1 +
 .../Binary/BinaryReflectiveSerializer.cs        |   44 +-
 .../Binary/TimestampAttribute.cs                |   40 +
 .../Cache/Query/SqlFieldsQuery.cs               |   29 +-
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |   31 +
 .../Impl/Binary/BinaryReflectiveActions.cs      |   89 +-
 .../BinaryReflectiveSerializerInternal.cs       |   10 +-
 .../Impl/Binary/BinaryUtils.cs                  |    4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |    3 +
 .../Apache.Ignite.Linq/CacheExtensions.cs       |   10 +
 .../Impl/CacheFieldsQueryExecutor.cs            |   42 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryable.cs   |    3 +-
 .../dotnet/Apache.Ignite.Linq/QueryOptions.cs   |   23 +
 .../processors/schedule/ScheduleFutureImpl.java |    2 +-
 modules/spark/pom.xml                           |    6 -
 .../spring/SpringTransactionManager.java        |   13 +
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |    2 +-
 .../GridSpringBeanSerializationSelfTest.java    |   15 +-
 .../GridSpringTransactionManagerSelfTest.java   |   45 +
 modules/web-console/frontend/app/app.js         |    4 +-
 .../activities-user-dialog.tpl.pug              |    1 +
 .../app/components/ignite-icon/directive.js     |   67 +
 .../app/components/ignite-icon/index.js         |   25 +
 .../app/components/ignite-icon/style.scss       |   21 +
 .../list-of-registered-users.tpl.pug            |    6 +-
 .../components/web-console-header/component.js  |   19 +-
 .../components/web-console-header/style.scss    |   39 +-
 .../components/web-console-header/template.pug  |   17 +-
 .../frontend/app/helpers/jade/mixins.pug        |   10 +-
 .../states/configuration/igfs/secondary.pug     |    2 +-
 .../frontend/app/primitives/btn/index.scss      |   58 +-
 .../app/primitives/datepicker/index.pug         |    2 +-
 .../frontend/app/primitives/dropdown/index.pug  |    7 +-
 .../frontend/app/primitives/dropdown/index.scss |   88 +-
 .../frontend/app/primitives/modal/index.scss    |   18 +-
 .../app/primitives/ui-grid-settings/index.scss  |   30 +-
 .../frontend/app/primitives/ui-grid/index.scss  |    6 +-
 .../frontend/gulpfile.babel.js/paths.js         |    1 +
 .../gulpfile.babel.js/webpack/common.js         |   10 +-
 modules/web-console/frontend/package.json       |    1 +
 .../frontend/public/images/icons/clock.svg      |    1 +
 .../frontend/public/images/icons/cross.svg      |    2 +-
 .../frontend/public/images/icons/csv.svg        |    1 +
 .../frontend/public/images/icons/download.svg   |    2 +
 .../frontend/public/images/icons/export.svg     |    1 -
 .../frontend/public/images/icons/gear.svg       |    2 +-
 .../frontend/public/images/icons/index.js       |   23 +
 .../frontend/public/images/icons/manual.svg     |    1 +
 .../frontend/public/stylesheets/style.scss      |   10 +-
 .../frontend/public/stylesheets/variables.scss  |    1 +
 modules/web-console/frontend/views/index.pug    |    1 +
 .../webapp2/META-INF/ignite-webapp-config.xml   |   15 +-
 .../ignite-int-max-values-onheap-config.xml     |    4 +-
 modules/yardstick/config/ignite-jdbc-config.xml |    4 +-
 parent/pom.xml                                  |    2 +-
 353 files changed, 10554 insertions(+), 5707 deletions(-)
----------------------------------------------------------------------



[18/39] ignite git commit: IGNITE-5284: Splitted IgniteH2Indexing into several classes. This closes #1999.

Posted by sb...@apache.org.
IGNITE-5284: Splitted IgniteH2Indexing into several classes. This closes #1999.


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 8c75e4de89496e1b80d854fbaa64d3bffa8193bd
Parents: d8eeea8
Author: devozerov <vo...@gridgain.com>
Authored: Wed May 24 15:38:14 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 24 15:38:14 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        |   39 +-
 .../query/h2/GridH2ResultSetIterator.java       |  191 --
 .../query/h2/H2ConnectionWrapper.java           |   67 +
 .../processors/query/h2/H2DatabaseType.java     |  161 ++
 .../processors/query/h2/H2FieldsIterator.java   |   50 +
 .../processors/query/h2/H2KeyValueIterator.java |   48 +
 .../query/h2/H2ResultSetIterator.java           |  191 ++
 .../processors/query/h2/H2RowDescriptor.java    |  479 +++++
 .../internal/processors/query/h2/H2Schema.java  |  135 ++
 .../processors/query/h2/H2SqlFieldMetadata.java |  111 +
 .../processors/query/h2/H2StatementCache.java   |   73 +
 .../processors/query/h2/H2TableDescriptor.java  |  345 ++++
 .../processors/query/h2/H2TableEngine.java      |   89 +
 .../query/h2/H2TwoStepCachedQuery.java          |   49 +
 .../query/h2/H2TwoStepCachedQueryKey.java       |  107 +
 .../internal/processors/query/h2/H2Utils.java   |  299 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 1941 ++----------------
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    6 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   21 +-
 20 files changed, 2413 insertions(+), 1993 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index e40c328..47b5ef4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -125,13 +125,13 @@ public class DmlStatementsProcessor {
      * @param cacheName Cache name.
      */
     public void onCacheStop(String cacheName) {
-        planCache.remove(cacheName);
+        planCache.remove(idx.schema(cacheName));
     }
 
     /**
      * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications.
      *
-     * @param cacheName Cache name.
+     * @param schema Schema.
      * @param stmt JDBC statement.
      * @param fieldsQry Original query.
      * @param loc Query locality flag.
@@ -140,14 +140,13 @@ public class DmlStatementsProcessor {
      * @return Update result (modified items count and failed keys).
      * @throws IgniteCheckedException if failed.
      */
-    private UpdateResult updateSqlFields(String cacheName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
-        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel)
-        throws IgniteCheckedException {
+    private UpdateResult updateSqlFields(String schema, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
+        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
         Object[] errKeys = null;
 
         long items = 0;
 
-        UpdatePlan plan = getPlanForStatement(cacheName, stmt, null);
+        UpdatePlan plan = getPlanForStatement(schema, stmt, null);
 
         GridCacheContext<?, ?> cctx = plan.tbl.rowDescriptor().context();
 
@@ -195,7 +194,7 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * @param cacheName Cache name.
+     * @param schema Schema.
      * @param stmt Prepared statement.
      * @param fieldsQry Initial query
      * @param cancel Query cancel.
@@ -203,9 +202,9 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String cacheName, PreparedStatement stmt,
+    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String schema, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, false, null, cancel);
+        UpdateResult res = updateSqlFields(schema, stmt, fieldsQry, false, null, cancel);
 
         QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
             (Collections.singletonList(res.cnt)), null, false);
@@ -217,7 +216,8 @@ public class DmlStatementsProcessor {
 
     /**
      * Execute DML statement on local cache.
-     * @param cacheName Cache name.
+     *
+     * @param schema Schema.
      * @param stmt Prepared statement.
      * @param fieldsQry Fields query.
      * @param filters Cache name and key filter.
@@ -226,10 +226,10 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    GridQueryFieldsResult updateSqlFieldsLocal(String cacheName, PreparedStatement stmt,
+    GridQueryFieldsResult updateSqlFieldsLocal(String schema, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel)
         throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, true, filters, cancel);
+        UpdateResult res = updateSqlFields(schema, stmt, fieldsQry, true, filters, cancel);
 
         return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
             new IgniteSingletonIterator(Collections.singletonList(res.cnt)));
@@ -333,7 +333,7 @@ public class DmlStatementsProcessor {
         Object[] failedKeys) throws IgniteCheckedException {
         Integer errKeysPos = null;
 
-        UpdatePlan plan = getPlanForStatement(cctx.name(), prepStmt, errKeysPos);
+        UpdatePlan plan = getPlanForStatement(idx.schema(cctx.name()), prepStmt, errKeysPos);
 
         if (plan.fastUpdateArgs != null) {
             assert F.isEmpty(failedKeys) && errKeysPos == null;
@@ -398,23 +398,22 @@ public class DmlStatementsProcessor {
     /**
      * Generate SELECT statements to retrieve data for modifications from and find fast UPDATE or DELETE args,
      * if available.
-     * @param cacheName Cache name.
+     *
+     * @param schema Schema.
      * @param prepStmt JDBC statement.
      * @return Update plan.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    private UpdatePlan getPlanForStatement(String cacheName, PreparedStatement prepStmt,
-        @Nullable Integer errKeysPos) throws IgniteCheckedException {
+    private UpdatePlan getPlanForStatement(String schema, PreparedStatement prepStmt, @Nullable Integer errKeysPos)
+        throws IgniteCheckedException {
         Prepared p = GridSqlQueryParser.prepared(prepStmt);
 
-        cacheName = F.isEmpty(cacheName) ? "default" : cacheName;
-
-        ConcurrentMap<String, UpdatePlan> cachePlans = planCache.get(cacheName);
+        ConcurrentMap<String, UpdatePlan> cachePlans = planCache.get(schema);
 
         if (cachePlans == null) {
             cachePlans = new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
 
-            cachePlans = U.firstNotNull(planCache.putIfAbsent(cacheName, cachePlans), cachePlans);
+            cachePlans = U.firstNotNull(planCache.putIfAbsent(schema, cachePlans), cachePlans);
         }
 
         // getSQL returns field value, so it's fast

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
deleted file mode 100644
index fed292a..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.h2;
-
-import java.lang.reflect.Field;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.NoSuchElementException;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
-import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.h2.jdbc.JdbcResultSet;
-import org.h2.result.ResultInterface;
-import org.h2.value.Value;
-
-/**
- * Iterator over result set.
- */
-public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAdapter<T> {
-    /** */
-    private static final Field RESULT_FIELD;
-
-    /**
-     * Initialize.
-     */
-    static {
-        try {
-            RESULT_FIELD = JdbcResultSet.class.getDeclaredField("result");
-
-            RESULT_FIELD.setAccessible(true);
-        }
-        catch (NoSuchFieldException e) {
-            throw new IllegalStateException("Check H2 version in classpath.", e);
-        }
-    }
-
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private final ResultInterface res;
-
-    /** */
-    private final ResultSet data;
-
-    /** */
-    protected final Object[] row;
-
-    /** */
-    private final boolean closeStmt;
-
-    /** */
-    private boolean hasRow;
-
-    /**
-     * @param data Data array.
-     * @param closeStmt If {@code true} closes result set statement when iterator is closed.
-     * @param needCpy {@code True} if need copy cache object's value.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt, boolean needCpy) throws IgniteCheckedException {
-        this.data = data;
-        this.closeStmt = closeStmt;
-
-        try {
-            res = needCpy ? (ResultInterface)RESULT_FIELD.get(data) : null;
-        }
-        catch (IllegalAccessException e) {
-            throw new IllegalStateException(e); // Must not happen.
-        }
-
-        if (data != null) {
-            try {
-                row = new Object[data.getMetaData().getColumnCount()];
-            }
-            catch (SQLException e) {
-                throw new IgniteCheckedException(e);
-            }
-        }
-        else
-            row = null;
-    }
-
-    /**
-     * @return {@code true} If next row was fetched successfully.
-     */
-    private boolean fetchNext() {
-        if (data == null)
-            return false;
-
-        try {
-            if (!data.next())
-                return false;
-
-            if (res != null) {
-                Value[] values = res.currentRow();
-
-                for (int c = 0; c < row.length; c++) {
-                    Value val = values[c];
-
-                    if (val instanceof GridH2ValueCacheObject) {
-                        GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)values[c];
-
-                        GridCacheContext cctx = valCacheObj.getCacheContext();
-
-                        row[c] = valCacheObj.getObject(cctx != null && cctx.needValueCopy());
-                    }
-                    else
-                        row[c] = val.getObject();
-                }
-            }
-            else {
-                for (int c = 0; c < row.length; c++)
-                    row[c] = data.getObject(c + 1);
-            }
-
-            return true;
-        }
-        catch (SQLException e) {
-            throw new IgniteSQLException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onHasNext() {
-        return hasRow || (hasRow = fetchNext());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("IteratorNextCanNotThrowNoSuchElementException")
-    @Override public T onNext() {
-        if (!hasNext())
-            throw new NoSuchElementException();
-
-        hasRow = false;
-
-        return createRow();
-    }
-
-    /**
-     * @return Row.
-     */
-    protected abstract T createRow();
-
-    /** {@inheritDoc} */
-    @Override public void onRemove() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() throws IgniteCheckedException {
-        if (data == null)
-            // Nothing to close.
-            return;
-
-        if (closeStmt) {
-            try {
-                U.closeQuiet(data.getStatement());
-            }
-            catch (SQLException e) {
-                throw new IgniteCheckedException(e);
-            }
-        }
-
-        U.closeQuiet(data);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridH2ResultSetIterator.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java
new file mode 100644
index 0000000..e180c9c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.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.query.h2;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.sql.Connection;
+
+/**
+ * Wrapper to store connection and flag is schema set or not.
+ */
+public class H2ConnectionWrapper {
+    /** */
+    private Connection conn;
+
+    /** */
+    private volatile String schema;
+
+    /**
+     * @param conn Connection to use.
+     */
+    H2ConnectionWrapper(Connection conn) {
+        this.conn = conn;
+    }
+
+    /**
+     * @return Schema name if schema is set, null otherwise.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @param schema Schema name set on this connection.
+     */
+    public void schema(@Nullable String schema) {
+        this.schema = schema;
+    }
+
+    /**
+     * @return Connection.
+     */
+    public Connection connection() {
+        return conn;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(H2ConnectionWrapper.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
new file mode 100644
index 0000000..47c7eb9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.h2.value.DataType;
+
+import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Enum that helps to map java types to database types.
+ */
+public enum H2DatabaseType {
+    /** */
+    INT("INT"),
+
+    /** */
+    BOOL("BOOL"),
+
+    /** */
+    TINYINT("TINYINT"),
+
+    /** */
+    SMALLINT("SMALLINT"),
+
+    /** */
+    BIGINT("BIGINT"),
+
+    /** */
+    DECIMAL("DECIMAL"),
+
+    /** */
+    DOUBLE("DOUBLE"),
+
+    /** */
+    REAL("REAL"),
+
+    /** */
+    TIME("TIME"),
+
+    /** */
+    TIMESTAMP("TIMESTAMP"),
+
+    /** */
+    DATE("DATE"),
+
+    /** */
+    VARCHAR("VARCHAR"),
+
+    /** */
+    CHAR("CHAR"),
+
+    /** */
+    BINARY("BINARY"),
+
+    /** */
+    UUID("UUID"),
+
+    /** */
+    ARRAY("ARRAY"),
+
+    /** */
+    GEOMETRY("GEOMETRY"),
+
+    /** */
+    OTHER("OTHER");
+
+    /** Map of Class to enum. */
+    private static final Map<Class<?>, H2DatabaseType> map = new HashMap<>();
+
+    /**
+     * Initialize map of DB types.
+     */
+    static {
+        map.put(int.class, INT);
+        map.put(Integer.class, INT);
+        map.put(boolean.class, BOOL);
+        map.put(Boolean.class, BOOL);
+        map.put(byte.class, TINYINT);
+        map.put(Byte.class, TINYINT);
+        map.put(short.class, SMALLINT);
+        map.put(Short.class, SMALLINT);
+        map.put(long.class, BIGINT);
+        map.put(Long.class, BIGINT);
+        map.put(BigDecimal.class, DECIMAL);
+        map.put(double.class, DOUBLE);
+        map.put(Double.class, DOUBLE);
+        map.put(float.class, REAL);
+        map.put(Float.class, REAL);
+        map.put(Time.class, TIME);
+        map.put(Timestamp.class, TIMESTAMP);
+        map.put(java.util.Date.class, TIMESTAMP);
+        map.put(java.sql.Date.class, DATE);
+        map.put(String.class, VARCHAR);
+        map.put(java.util.UUID.class, UUID);
+        map.put(byte[].class, BINARY);
+    }
+
+    /** */
+    private final String dbType;
+
+    /**
+     * Constructs new instance.
+     *
+     * @param dbType DB type name.
+     */
+    H2DatabaseType(String dbType) {
+        this.dbType = dbType;
+    }
+
+    /**
+     * Resolves enum by class.
+     *
+     * @param cls Class.
+     * @return Enum value.
+     */
+    public static H2DatabaseType fromClass(Class<?> cls) {
+        H2DatabaseType res = map.get(cls);
+
+        if (res != null)
+            return res;
+
+        if (DataType.isGeometryClass(cls))
+            return GEOMETRY;
+
+        return cls.isArray() && !cls.getComponentType().isPrimitive() ? ARRAY : OTHER;
+    }
+
+    /**
+     * Gets DB type name.
+     *
+     * @return DB type name.
+     */
+    public String dBTypeAsString() {
+        return dbType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(H2DatabaseType.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java
new file mode 100644
index 0000000..f300c3f
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.IgniteCheckedException;
+
+import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Special field set iterator based on database result set.
+ */
+public class H2FieldsIterator extends H2ResultSetIterator<List<?>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param data Data.
+     * @throws IgniteCheckedException If failed.
+     */
+    public H2FieldsIterator(ResultSet data) throws IgniteCheckedException {
+        super(data, false, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected List<?> createRow() {
+        ArrayList<Object> res = new ArrayList<>(row.length);
+
+        Collections.addAll(res, row);
+
+        return res;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java
new file mode 100644
index 0000000..2088e44
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import java.sql.ResultSet;
+
+/**
+ * Special key/value iterator based on database result set.
+ */
+public class H2KeyValueIterator<K, V> extends H2ResultSetIterator<IgniteBiTuple<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param data Data array.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected H2KeyValueIterator(ResultSet data) throws IgniteCheckedException {
+        super(data, false, true);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteBiTuple<K, V> createRow() {
+        K key = (K)row[0];
+        V val = (V)row[1];
+
+        return new IgniteBiTuple<>(key, val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java
new file mode 100644
index 0000000..494f069
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import java.lang.reflect.Field;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
+import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.h2.jdbc.JdbcResultSet;
+import org.h2.result.ResultInterface;
+import org.h2.value.Value;
+
+/**
+ * Iterator over result set.
+ */
+public abstract class H2ResultSetIterator<T> extends GridCloseableIteratorAdapter<T> {
+    /** */
+    private static final Field RESULT_FIELD;
+
+    /**
+     * Initialize.
+     */
+    static {
+        try {
+            RESULT_FIELD = JdbcResultSet.class.getDeclaredField("result");
+
+            RESULT_FIELD.setAccessible(true);
+        }
+        catch (NoSuchFieldException e) {
+            throw new IllegalStateException("Check H2 version in classpath.", e);
+        }
+    }
+
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final ResultInterface res;
+
+    /** */
+    private final ResultSet data;
+
+    /** */
+    protected final Object[] row;
+
+    /** */
+    private final boolean closeStmt;
+
+    /** */
+    private boolean hasRow;
+
+    /**
+     * @param data Data array.
+     * @param closeStmt If {@code true} closes result set statement when iterator is closed.
+     * @param needCpy {@code True} if need copy cache object's value.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected H2ResultSetIterator(ResultSet data, boolean closeStmt, boolean needCpy) throws IgniteCheckedException {
+        this.data = data;
+        this.closeStmt = closeStmt;
+
+        try {
+            res = needCpy ? (ResultInterface)RESULT_FIELD.get(data) : null;
+        }
+        catch (IllegalAccessException e) {
+            throw new IllegalStateException(e); // Must not happen.
+        }
+
+        if (data != null) {
+            try {
+                row = new Object[data.getMetaData().getColumnCount()];
+            }
+            catch (SQLException e) {
+                throw new IgniteCheckedException(e);
+            }
+        }
+        else
+            row = null;
+    }
+
+    /**
+     * @return {@code true} If next row was fetched successfully.
+     */
+    private boolean fetchNext() {
+        if (data == null)
+            return false;
+
+        try {
+            if (!data.next())
+                return false;
+
+            if (res != null) {
+                Value[] values = res.currentRow();
+
+                for (int c = 0; c < row.length; c++) {
+                    Value val = values[c];
+
+                    if (val instanceof GridH2ValueCacheObject) {
+                        GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)values[c];
+
+                        GridCacheContext cctx = valCacheObj.getCacheContext();
+
+                        row[c] = valCacheObj.getObject(cctx != null && cctx.needValueCopy());
+                    }
+                    else
+                        row[c] = val.getObject();
+                }
+            }
+            else {
+                for (int c = 0; c < row.length; c++)
+                    row[c] = data.getObject(c + 1);
+            }
+
+            return true;
+        }
+        catch (SQLException e) {
+            throw new IgniteSQLException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onHasNext() {
+        return hasRow || (hasRow = fetchNext());
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("IteratorNextCanNotThrowNoSuchElementException")
+    @Override public T onNext() {
+        if (!hasNext())
+            throw new NoSuchElementException();
+
+        hasRow = false;
+
+        return createRow();
+    }
+
+    /**
+     * @return Row.
+     */
+    protected abstract T createRow();
+
+    /** {@inheritDoc} */
+    @Override public void onRemove() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() throws IgniteCheckedException {
+        if (data == null)
+            // Nothing to close.
+            return;
+
+        if (closeStmt) {
+            try {
+                U.closeQuiet(data.getStatement());
+            }
+            catch (SQLException e) {
+                throw new IgniteCheckedException(e);
+            }
+        }
+
+        U.closeQuiet(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(H2ResultSetIterator.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
new file mode 100644
index 0000000..6f5ce3e
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
@@ -0,0 +1,479 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+import org.h2.message.DbException;
+import org.h2.result.SearchRow;
+import org.h2.result.SimpleRow;
+import org.h2.value.DataType;
+import org.h2.value.Value;
+import org.h2.value.ValueArray;
+import org.h2.value.ValueBoolean;
+import org.h2.value.ValueByte;
+import org.h2.value.ValueBytes;
+import org.h2.value.ValueDate;
+import org.h2.value.ValueDecimal;
+import org.h2.value.ValueDouble;
+import org.h2.value.ValueFloat;
+import org.h2.value.ValueGeometry;
+import org.h2.value.ValueInt;
+import org.h2.value.ValueJavaObject;
+import org.h2.value.ValueLong;
+import org.h2.value.ValueNull;
+import org.h2.value.ValueShort;
+import org.h2.value.ValueString;
+import org.h2.value.ValueTime;
+import org.h2.value.ValueTimestamp;
+import org.h2.value.ValueUuid;
+import org.jetbrains.annotations.Nullable;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.DEFAULT_COLUMNS_COUNT;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VAL_COL;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VER_COL;
+
+/**
+ * Row descriptor.
+ */
+public class H2RowDescriptor implements GridH2RowDescriptor {
+    /** Indexing SPI. */
+    private final IgniteH2Indexing idx;
+
+    /** */
+    private final GridQueryTypeDescriptor type;
+
+    /** */
+    private final String[] fields;
+
+    /** */
+    private final int[] fieldTypes;
+
+    /** */
+    private final int keyType;
+
+    /** */
+    private final int valType;
+
+    /** */
+    private final H2Schema schema;
+
+    /** */
+    private final GridUnsafeGuard guard;
+
+    /** */
+    private final boolean snapshotableIdx;
+
+    /** */
+    private final GridQueryProperty[] props;
+
+    /** Id of user-defined key column */
+    private final int keyAliasColumnId;
+
+    /** Id of user-defined value column */
+    private final int valueAliasColumnId;
+
+    /**
+     * @param type Type descriptor.
+     * @param schema Schema.
+     */
+    H2RowDescriptor(IgniteH2Indexing idx, GridQueryTypeDescriptor type, H2Schema schema) {
+        assert type != null;
+        assert schema != null;
+
+        this.idx = idx;
+        this.type = type;
+        this.schema = schema;
+
+        guard = schema.offheap() == null ? null : new GridUnsafeGuard();
+
+        Map<String, Class<?>> allFields = new LinkedHashMap<>();
+
+        allFields.putAll(type.fields());
+
+        fields = allFields.keySet().toArray(new String[allFields.size()]);
+
+        fieldTypes = new int[fields.length];
+
+        Class[] classes = allFields.values().toArray(new Class[fields.length]);
+
+        for (int i = 0; i < fieldTypes.length; i++)
+            fieldTypes[i] = DataType.getTypeFromClass(classes[i]);
+
+        keyType = DataType.getTypeFromClass(type.keyClass());
+        valType = DataType.getTypeFromClass(type.valueClass());
+
+        props = new GridQueryProperty[fields.length];
+
+        for (int i = 0; i < fields.length; i++) {
+            GridQueryProperty p = type.property(fields[i]);
+
+            assert p != null : fields[i];
+
+            props[i] = p;
+        }
+
+        final List<String> fieldsList = Arrays.asList(fields);
+        keyAliasColumnId = (type.keyFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.keyFieldName()) : -1;
+        valueAliasColumnId = (type.valueFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.valueFieldName()) : -1;
+
+        // Index is not snapshotable in db-x.
+        snapshotableIdx = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteH2Indexing indexing() {
+        return idx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridQueryTypeDescriptor type() {
+        return type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheContext<?, ?> context() {
+        return schema.cacheContext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheConfiguration configuration() {
+        return schema.cacheContext().config();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridUnsafeGuard guard() {
+        return guard;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cache(GridH2Row row) {
+        long ptr = row.pointer();
+
+        assert ptr > 0 : ptr;
+
+        schema.rowCache().put(ptr, row);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void uncache(long ptr) {
+        schema.rowCache().remove(ptr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridUnsafeMemory memory() {
+        return schema.offheap();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override public Value wrap(Object obj, int type) throws IgniteCheckedException {
+        assert obj != null;
+
+        if (obj instanceof CacheObject) { // Handle cache object.
+            CacheObject co = (CacheObject)obj;
+
+            if (type == Value.JAVA_OBJECT)
+                return new GridH2ValueCacheObject(idx.cacheContext(schema.cacheName()), co);
+
+            obj = co.value(idx.objectContext(schema.cacheName()), false);
+        }
+
+        switch (type) {
+            case Value.BOOLEAN:
+                return ValueBoolean.get((Boolean)obj);
+            case Value.BYTE:
+                return ValueByte.get((Byte)obj);
+            case Value.SHORT:
+                return ValueShort.get((Short)obj);
+            case Value.INT:
+                return ValueInt.get((Integer)obj);
+            case Value.FLOAT:
+                return ValueFloat.get((Float)obj);
+            case Value.LONG:
+                return ValueLong.get((Long)obj);
+            case Value.DOUBLE:
+                return ValueDouble.get((Double)obj);
+            case Value.UUID:
+                UUID uuid = (UUID)obj;
+                return ValueUuid.get(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits());
+            case Value.DATE:
+                return ValueDate.get((Date)obj);
+            case Value.TIME:
+                return ValueTime.get((Time)obj);
+            case Value.TIMESTAMP:
+                if (obj instanceof java.util.Date && !(obj instanceof Timestamp))
+                    obj = new Timestamp(((java.util.Date)obj).getTime());
+
+                return ValueTimestamp.get((Timestamp)obj);
+            case Value.DECIMAL:
+                return ValueDecimal.get((BigDecimal)obj);
+            case Value.STRING:
+                return ValueString.get(obj.toString());
+            case Value.BYTES:
+                return ValueBytes.get((byte[])obj);
+            case Value.JAVA_OBJECT:
+                return ValueJavaObject.getNoCopy(obj, null, null);
+            case Value.ARRAY:
+                Object[] arr = (Object[])obj;
+
+                Value[] valArr = new Value[arr.length];
+
+                for (int i = 0; i < arr.length; i++) {
+                    Object o = arr[i];
+
+                    valArr[i] = o == null ? ValueNull.INSTANCE : wrap(o, DataType.getTypeFromClass(o.getClass()));
+                }
+
+                return ValueArray.get(valArr);
+
+            case Value.GEOMETRY:
+                return ValueGeometry.getFromGeometry(obj);
+        }
+
+        throw new IgniteCheckedException("Failed to wrap value[type=" + type + ", value=" + obj + "]");
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridH2Row createRow(KeyCacheObject key, int partId, @Nullable CacheObject val,
+        GridCacheVersion ver,
+        long expirationTime) throws IgniteCheckedException {
+        GridH2Row row;
+
+        try {
+            if (val == null) // Only can happen for remove operation, can create simple search row.
+                row = GridH2RowFactory.create(wrap(key, keyType));
+            else
+                row = schema.offheap() == null ?
+                    new GridH2KeyValueRowOnheap(this, key, keyType, val, valType, ver, expirationTime) :
+                    new GridH2KeyValueRowOffheap(this, key, keyType, val, valType, ver, expirationTime);
+        }
+        catch (ClassCastException e) {
+            throw new IgniteCheckedException("Failed to convert key to SQL type. " +
+                "Please make sure that you always store each value type with the same key type " +
+                "or configure key type as common super class for all actual keys for this value type.", e);
+        }
+
+        GridCacheContext cctx = idx.cacheContext(schema.cacheName());
+
+        if (cctx.offheapIndex()) {
+            row.ver = ver;
+
+            row.key = key;
+            row.val = val;
+            row.partId = partId;
+        }
+
+        return row;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int valueType() {
+        return valType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int fieldsCount() {
+        return fields.length;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int fieldType(int col) {
+        return fieldTypes[col];
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object columnValue(Object key, Object val, int col) {
+        try {
+            return props[col].value(key, val);
+        }
+        catch (IgniteCheckedException e) {
+            throw DbException.convert(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setColumnValue(Object key, Object val, Object colVal, int col) {
+        try {
+            props[col].setValue(key, val, colVal);
+        }
+        catch (IgniteCheckedException e) {
+            throw DbException.convert(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isColumnKeyProperty(int col) {
+        return props[col].key();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridH2KeyValueRowOffheap createPointer(long ptr) {
+        GridH2KeyValueRowOffheap row = (GridH2KeyValueRowOffheap)schema.rowCache().get(ptr);
+
+        if (row != null) {
+            assert row.pointer() == ptr : ptr + " " + row.pointer();
+
+            return row;
+        }
+
+        return new GridH2KeyValueRowOffheap(this, ptr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridH2Row cachedRow(long link) {
+        return schema.rowCache().get(link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean snapshotableIndex() {
+        return snapshotableIdx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isKeyColumn(int columnId) {
+        assert columnId >= 0;
+        return columnId == KEY_COL || columnId == keyAliasColumnId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isValueColumn(int columnId) {
+        assert columnId >= 0;
+        return columnId == VAL_COL || columnId == valueAliasColumnId;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("RedundantIfStatement")
+    @Override public boolean isKeyValueOrVersionColumn(int columnId) {
+        assert columnId >= 0;
+
+        if (columnId < DEFAULT_COLUMNS_COUNT)
+            return true;
+
+        if (columnId == keyAliasColumnId)
+            return true;
+
+        if (columnId == valueAliasColumnId)
+            return true;
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean checkKeyIndexCondition(int masks[], int mask) {
+        assert masks != null;
+        assert masks.length > 0;
+
+        if (keyAliasColumnId < 0)
+            return (masks[KEY_COL] & mask) != 0;
+        else
+            return (masks[KEY_COL] & mask) != 0 || (masks[keyAliasColumnId] & mask) != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initValueCache(Value valCache[], Value key, Value value, Value version) {
+        assert valCache != null;
+        assert valCache.length > 0;
+
+        valCache[KEY_COL] = key;
+        valCache[VAL_COL] = value;
+        valCache[VER_COL] = version;
+
+        if (keyAliasColumnId > 0)
+            valCache[keyAliasColumnId] = key;
+
+        if (valueAliasColumnId > 0)
+            valCache[valueAliasColumnId] = value;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SearchRow prepareProxyIndexRow(SearchRow row) {
+        if (row == null)
+            return null;
+
+        Value[] data = new Value[row.getColumnCount()];
+        for (int idx = 0; idx < data.length; idx++)
+            data[idx] = row.getValue(idx);
+
+        copyAliasColumnData(data, KEY_COL, keyAliasColumnId);
+        copyAliasColumnData(data, VAL_COL, valueAliasColumnId);
+
+        return new SimpleRow(data);
+    }
+
+    /**
+     * Copies data between original and alias columns
+     *
+     * @param data Array of values.
+     * @param colId Original column id.
+     * @param aliasColId Alias column id.
+     */
+    private void copyAliasColumnData(Value[] data, int colId, int aliasColId) {
+        if (aliasColId <= 0)
+            return;
+
+        if (data[aliasColId] == null && data[colId] != null)
+            data[aliasColId] = data[colId];
+
+        if (data[colId] == null && data[aliasColId] != null)
+            data[colId] = data[aliasColId];
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getAlternativeColumnId(int colId) {
+        if (keyAliasColumnId > 0) {
+            if (colId == KEY_COL)
+                return keyAliasColumnId;
+            else if (colId == keyAliasColumnId)
+                return KEY_COL;
+        }
+        if (valueAliasColumnId > 0) {
+            if (colId == VAL_COL)
+                return valueAliasColumnId;
+            else if (colId == valueAliasColumnId)
+                return VAL_COL;
+        }
+
+        return colId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
new file mode 100644
index 0000000..603a0c1
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.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.query.h2;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+import org.h2.mvstore.cache.CacheLongKeyLIRS;
+import org.jsr166.ConcurrentHashMap8;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Database schema object.
+ */
+public class H2Schema {
+    /** */
+    private final String cacheName;
+
+    /** */
+    private final String schemaName;
+
+    /** */
+    private final GridUnsafeMemory offheap = null;
+
+    /** */
+    private final ConcurrentMap<String, H2TableDescriptor> tbls = new ConcurrentHashMap8<>();
+
+    /** Cache for deserialized offheap rows. */
+    private final CacheLongKeyLIRS<GridH2Row> rowCache;
+
+    /** */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    private final CacheConfiguration<?, ?> ccfg;
+
+    /**
+     * @param cacheName Cache name.
+     * @param schemaName Schema name.
+     * @param cctx Cache context.
+     * @param ccfg Cache configuration.
+     */
+    H2Schema(String cacheName, String schemaName, GridCacheContext<?, ?> cctx,
+        CacheConfiguration<?, ?> ccfg) {
+        this.cacheName = cacheName;
+        this.cctx = cctx;
+        this.schemaName = schemaName;
+        this.ccfg = ccfg;
+
+        rowCache = null;
+    }
+
+    /**
+     * @return Cache context.
+     */
+    public GridCacheContext cacheContext() {
+        return cctx;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Unsafe memory.
+     */
+    public GridUnsafeMemory offheap() {
+        return offheap;
+    }
+
+    /**
+     * @return Row cache.
+     */
+    public CacheLongKeyLIRS<GridH2Row> rowCache() {
+        return rowCache;
+    }
+
+    /**
+     * @return Tables.
+     */
+    public Map<String, H2TableDescriptor> tables() {
+        return tbls;
+    }
+
+    /**
+     * @param tbl Table descriptor.
+     */
+    public void add(H2TableDescriptor tbl) {
+        if (tbls.putIfAbsent(tbl.typeName(), tbl) != null)
+            throw new IllegalStateException("Table already registered: " + tbl.fullTableName());
+    }
+
+    /**
+     * @return Escape all.
+     */
+    public boolean escapeAll() {
+        return ccfg.isSqlEscapeAll();
+    }
+
+    /**
+     * Called after the schema was dropped.
+     */
+    public void onDrop() {
+        for (H2TableDescriptor tblDesc : tbls.values())
+            tblDesc.onDrop();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SqlFieldMetadata.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SqlFieldMetadata.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SqlFieldMetadata.java
new file mode 100644
index 0000000..46aa1fc
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SqlFieldMetadata.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Field descriptor.
+ */
+public class H2SqlFieldMetadata implements GridQueryFieldMetadata {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Schema name. */
+    private String schemaName;
+
+    /** Type name. */
+    private String typeName;
+
+    /** Name. */
+    private String name;
+
+    /** Type. */
+    private String type;
+
+    /**
+     * Required by {@link Externalizable}.
+     */
+    public H2SqlFieldMetadata() {
+        // No-op
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param typeName Type name.
+     * @param name Name.
+     * @param type Type.
+     */
+    H2SqlFieldMetadata(@Nullable String schemaName, @Nullable String typeName, String name, String type) {
+        assert name != null && type != null : schemaName + " | " + typeName + " | " + name + " | " + type;
+
+        this.schemaName = schemaName;
+        this.typeName = typeName;
+        this.name = name;
+        this.type = type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String schemaName() {
+        return schemaName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String typeName() {
+        return typeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String fieldName() {
+        return name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String fieldTypeName() {
+        return type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, schemaName);
+        U.writeString(out, typeName);
+        U.writeString(out, name);
+        U.writeString(out, type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        schemaName = U.readString(in);
+        typeName = U.readString(in);
+        name = U.readString(in);
+        type = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(H2SqlFieldMetadata.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java
new file mode 100644
index 0000000..d395112
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.sql.PreparedStatement;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Statement cache.
+ */
+public class H2StatementCache extends LinkedHashMap<String, PreparedStatement> {
+    /** */
+    private int size;
+
+    /** Last usage. */
+    private volatile long lastUsage;
+
+    /**
+     * @param size Size.
+     */
+    H2StatementCache(int size) {
+        super(size, (float)0.75, true);
+
+        this.size = size;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean removeEldestEntry(Map.Entry<String, PreparedStatement> eldest) {
+        boolean rmv = size() > size;
+
+        if (rmv) {
+            PreparedStatement stmt = eldest.getValue();
+
+            U.closeQuiet(stmt);
+        }
+
+        return rmv;
+    }
+
+    /**
+     * The timestamp of the last usage of the cache.
+     *
+     * @return last usage timestamp
+     */
+    public long lastUsage() {
+        return lastUsage;
+    }
+
+    /**
+     * Updates the {@link #lastUsage} timestamp by current time.
+     */
+    public void updateLastUsage() {
+        lastUsage = U.currentTimeMillis();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
new file mode 100644
index 0000000..a9548aa
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.h2.database.H2PkHashIndex;
+import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2SystemIndexFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.GridLuceneIndex;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.h2.index.Index;
+import org.h2.result.SortOrder;
+import org.h2.table.Column;
+import org.h2.table.IndexColumn;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
+
+/**
+ * Information about table in database.
+ */
+public class H2TableDescriptor implements GridH2SystemIndexFactory {
+    /** Indexing. */
+    private final IgniteH2Indexing idx;
+
+    /** */
+    private final String fullTblName;
+
+    /** */
+    private final GridQueryTypeDescriptor type;
+
+    /** */
+    private final H2Schema schema;
+
+    /** */
+    private GridH2Table tbl;
+
+    /** */
+    private GridLuceneIndex luceneIdx;
+
+    /** */
+    private H2PkHashIndex pkHashIdx;
+
+    /**
+     * Constructor.
+     *
+     * @param idx Indexing.
+     * @param schema Schema.
+     * @param type Type descriptor.
+     */
+    H2TableDescriptor(IgniteH2Indexing idx, H2Schema schema, GridQueryTypeDescriptor type) {
+        this.idx = idx;
+        this.type = type;
+        this.schema = schema;
+
+        String tblName = H2Utils.escapeName(type.tableName(), schema.escapeAll());
+
+        fullTblName = schema.schemaName() + "." + tblName;
+    }
+
+    /**
+     * @return Primary key hash index.
+     */
+    H2PkHashIndex primaryKeyHashIndex() {
+        return pkHashIdx;
+    }
+
+    /**
+     * @return Table.
+     */
+    public GridH2Table table() {
+        return tbl;
+    }
+
+    /**
+     * @param tbl Table.
+     */
+    public void table(GridH2Table tbl) {
+        this.tbl = tbl;
+    }
+
+    /**
+     * @return Schema.
+     */
+    public H2Schema schema() {
+        return schema;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schema.schemaName();
+    }
+
+    /**
+     * @return Database full table name.
+     */
+    String fullTableName() {
+        return fullTblName;
+    }
+
+    /**
+     * @return type name.
+     */
+    String typeName() {
+        return type.name();
+    }
+
+    /**
+     * @return Type.
+     */
+    GridQueryTypeDescriptor type() {
+        return type;
+    }
+
+    /**
+     * @return Lucene index.
+     */
+    GridLuceneIndex luceneIndex() {
+        return luceneIdx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(H2TableDescriptor.class, this);
+    }
+
+    /**
+     * Create H2 row factory.
+     *
+     * @param rowDesc Row descriptor.
+     * @return H2 row factory.
+     */
+    H2RowFactory rowFactory(GridH2RowDescriptor rowDesc) {
+        GridCacheContext cctx = schema.cacheContext();
+
+        if (cctx.affinityNode() && cctx.offheapIndex())
+            return new H2RowFactory(rowDesc, cctx);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ArrayList<Index> createSystemIndexes(GridH2Table tbl) {
+        ArrayList<Index> idxs = new ArrayList<>();
+
+        IndexColumn keyCol = tbl.indexColumn(KEY_COL, SortOrder.ASCENDING);
+        IndexColumn affCol = tbl.getAffinityKeyColumn();
+
+        if (affCol != null && H2Utils.equals(affCol, keyCol))
+            affCol = null;
+
+        GridH2RowDescriptor desc = tbl.rowDescriptor();
+
+        Index hashIdx = createHashIndex(
+            schema,
+            tbl,
+            "_key_PK_hash",
+            H2Utils.treeIndexColumns(desc, new ArrayList<IndexColumn>(2), keyCol, affCol)
+        );
+
+        if (hashIdx != null)
+            idxs.add(hashIdx);
+
+        // Add primary key index.
+        Index pkIdx = idx.createSortedIndex(
+            schema,
+            "_key_PK",
+            tbl,
+            true,
+            H2Utils.treeIndexColumns(desc, new ArrayList<IndexColumn>(2), keyCol, affCol),
+            -1
+        );
+
+        idxs.add(pkIdx);
+
+        if (type().valueClass() == String.class) {
+            try {
+                luceneIdx = new GridLuceneIndex(idx.kernalContext(), schema.offheap(), schema.cacheName(), type);
+            }
+            catch (IgniteCheckedException e1) {
+                throw new IgniteException(e1);
+            }
+        }
+
+        boolean affIdxFound = false;
+
+        GridQueryIndexDescriptor textIdx = type.textIndex();
+
+        if (textIdx != null) {
+            try {
+                luceneIdx = new GridLuceneIndex(idx.kernalContext(), schema.offheap(), schema.cacheName(), type);
+            }
+            catch (IgniteCheckedException e1) {
+                throw new IgniteException(e1);
+            }
+        }
+
+        // Locate index where affinity column is first (if any).
+        if (affCol != null) {
+            for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
+                if (idxDesc.type() != QueryIndexType.SORTED)
+                    continue;
+
+                String firstField = idxDesc.fields().iterator().next();
+
+                String firstFieldName =
+                    schema.escapeAll() ? firstField : H2Utils.escapeName(firstField, false).toUpperCase();
+
+                Column col = tbl.getColumn(firstFieldName);
+
+                IndexColumn idxCol = tbl.indexColumn(col.getColumnId(),
+                    idxDesc.descending(firstField) ? SortOrder.DESCENDING : SortOrder.ASCENDING);
+
+                affIdxFound |= H2Utils.equals(idxCol, affCol);
+            }
+        }
+
+        // Add explicit affinity key index if nothing alike was found.
+        if (affCol != null && !affIdxFound) {
+            idxs.add(idx.createSortedIndex(schema, "AFFINITY_KEY", tbl, false,
+                H2Utils.treeIndexColumns(desc, new ArrayList<IndexColumn>(2), affCol, keyCol), -1));
+        }
+
+        return idxs;
+    }
+
+    /**
+     * Get collection of user indexes.
+     *
+     * @return User indexes.
+     */
+    public Collection<GridH2IndexBase> createUserIndexes() {
+        assert tbl != null;
+
+        ArrayList<GridH2IndexBase> res = new ArrayList<>();
+
+        for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
+            GridH2IndexBase idx = createUserIndex(idxDesc);
+
+            res.add(idx);
+        }
+
+        return res;
+    }
+
+    /**
+     * Create user index.
+     *
+     * @param idxDesc Index descriptor.
+     * @return Index.
+     */
+    public GridH2IndexBase createUserIndex(GridQueryIndexDescriptor idxDesc) {
+        String name = schema.escapeAll() ? idxDesc.name() : H2Utils.escapeName(idxDesc.name(), false).toUpperCase();
+
+        IndexColumn keyCol = tbl.indexColumn(KEY_COL, SortOrder.ASCENDING);
+        IndexColumn affCol = tbl.getAffinityKeyColumn();
+
+        List<IndexColumn> cols = new ArrayList<>(idxDesc.fields().size() + 2);
+
+        boolean escapeAll = schema.escapeAll();
+
+        for (String field : idxDesc.fields()) {
+            String fieldName = escapeAll ? field : H2Utils.escapeName(field, false).toUpperCase();
+
+            Column col = tbl.getColumn(fieldName);
+
+            cols.add(tbl.indexColumn(col.getColumnId(),
+                idxDesc.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
+        }
+
+        GridH2RowDescriptor desc = tbl.rowDescriptor();
+        if (idxDesc.type() == QueryIndexType.SORTED) {
+            cols = H2Utils.treeIndexColumns(desc, cols, keyCol, affCol);
+            return idx.createSortedIndex(schema, name, tbl, false, cols, idxDesc.inlineSize());
+        }
+        else if (idxDesc.type() == QueryIndexType.GEOSPATIAL) {
+            return H2Utils.createSpatialIndex(tbl, name, cols.toArray(new IndexColumn[cols.size()]));
+        }
+
+        throw new IllegalStateException("Index type: " + idxDesc.type());
+    }
+
+    /**
+     * Create hash index.
+     *
+     * @param schema Schema.
+     * @param tbl Table.
+     * @param idxName Index name.
+     * @param cols Columns.
+     * @return Index.
+     */
+    private Index createHashIndex(H2Schema schema, GridH2Table tbl, String idxName, List<IndexColumn> cols) {
+        GridCacheContext cctx = schema.cacheContext();
+
+        if (cctx.affinityNode() && cctx.offheapIndex()) {
+            assert pkHashIdx == null : pkHashIdx;
+
+            pkHashIdx = new H2PkHashIndex(cctx, tbl, idxName, cols);
+
+            return pkHashIdx;
+        }
+
+        return null;
+    }
+
+    /**
+     * Handle drop.
+     */
+    void onDrop() {
+        idx.removeDataTable(tbl);
+
+        tbl.destroy();
+
+        U.closeQuiet(luceneIdx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
new file mode 100644
index 0000000..4cf5166
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.h2.api.TableEngine;
+import org.h2.command.ddl.CreateTableData;
+import org.h2.table.TableBase;
+import org.jetbrains.annotations.Nullable;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+/**
+ * H2 Table engine.
+ */
+public class H2TableEngine implements TableEngine {
+    /** */
+    private static GridH2RowDescriptor rowDesc0;
+
+    /** */
+    private static H2RowFactory rowFactory0;
+
+    /** */
+    private static H2TableDescriptor tblDesc0;
+
+    /** */
+    private static GridH2Table resTbl0;
+
+    /**
+     * Creates table using given connection, DDL clause for given type descriptor and list of indexes.
+     *
+     * @param conn Connection.
+     * @param sql DDL clause.
+     * @param rowDesc Row descriptor.
+     * @param rowFactory Row factory.
+     * @param tblDesc Table descriptor.
+     * @throws SQLException If failed.
+     * @return Created table.
+     */
+    public static synchronized GridH2Table createTable(Connection conn, String sql,
+        @Nullable GridH2RowDescriptor rowDesc, H2RowFactory rowFactory, H2TableDescriptor tblDesc)
+        throws SQLException {
+        rowDesc0 = rowDesc;
+        rowFactory0 = rowFactory;
+        tblDesc0 = tblDesc;
+
+        try {
+            try (Statement s = conn.createStatement()) {
+                s.execute(sql + " engine \"" + H2TableEngine.class.getName() + "\"");
+            }
+
+            tblDesc.table(resTbl0);
+
+            return resTbl0;
+        }
+        finally {
+            resTbl0 = null;
+            tblDesc0 = null;
+            rowFactory0 = null;
+            rowDesc0 = null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public TableBase createTable(CreateTableData createTblData) {
+        resTbl0 = new GridH2Table(createTblData, rowDesc0, rowFactory0, tblDesc0, tblDesc0.schema().cacheName());
+
+        return resTbl0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
new file mode 100644
index 0000000..dd1b44c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.List;
+
+/**
+ * Cached two-step query.
+ */
+public class H2TwoStepCachedQuery {
+    /** */
+    final List<GridQueryFieldMetadata> meta;
+
+    /** */
+    final GridCacheTwoStepQuery twoStepQry;
+
+    /**
+     * @param meta Fields metadata.
+     * @param twoStepQry Query.
+     */
+    public H2TwoStepCachedQuery(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
+        this.meta = meta;
+        this.twoStepQry = twoStepQry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(H2TwoStepCachedQuery.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
new file mode 100644
index 0000000..1452a83
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+/**
+ * Key for cached two-step query.
+ */
+public class H2TwoStepCachedQueryKey {
+    /** */
+    private final String cacheName;
+
+    /** */
+    private final String sql;
+
+    /** */
+    private final boolean grpByCollocated;
+
+    /** */
+    private final boolean distributedJoins;
+
+    /** */
+    private final boolean enforceJoinOrder;
+
+    /** */
+    private final boolean isLocal;
+
+    /**
+     * @param cacheName Cache name.
+     * @param sql Sql.
+     * @param grpByCollocated Collocated GROUP BY.
+     * @param distributedJoins Distributed joins enabled.
+     * @param enforceJoinOrder Enforce join order of tables.
+     * @param isLocal Query is local flag.
+     */
+    H2TwoStepCachedQueryKey(String cacheName,
+        String sql,
+        boolean grpByCollocated,
+        boolean distributedJoins,
+        boolean enforceJoinOrder,
+        boolean isLocal) {
+        this.cacheName = cacheName;
+        this.sql = sql;
+        this.grpByCollocated = grpByCollocated;
+        this.distributedJoins = distributedJoins;
+        this.enforceJoinOrder = enforceJoinOrder;
+        this.isLocal = isLocal;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        H2TwoStepCachedQueryKey that = (H2TwoStepCachedQueryKey)o;
+
+        if (grpByCollocated != that.grpByCollocated)
+            return false;
+
+        if (distributedJoins != that.distributedJoins)
+            return false;
+
+        if (enforceJoinOrder != that.enforceJoinOrder)
+            return false;
+
+        if (cacheName != null ? !cacheName.equals(that.cacheName) : that.cacheName != null)
+            return false;
+
+        return isLocal == that.isLocal && sql.equals(that.sql);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = cacheName != null ? cacheName.hashCode() : 0;
+        res = 31 * res + sql.hashCode();
+        res = 31 * res + (grpByCollocated ? 1 : 0);
+        res = res + (distributedJoins ? 2 : 0);
+        res = res + (enforceJoinOrder ? 4 : 0);
+        res = res + (isLocal ? 8 : 0);
+
+        return res;
+    }
+}


[10/39] ignite git commit: master - BPlusTree: compare with lvl

Posted by sb...@apache.org.
master - BPlusTree: compare with lvl


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: ff813891dffb24021f4f7b744cf30d58d919dc42
Parents: 33cb5e8
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed May 24 08:52:24 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed May 24 08:52:24 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 64 +++++++++++++-------
 1 file changed, 41 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ff813891/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index a4c09d5..4d2110c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -253,11 +253,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             int cnt = io.getCount(pageAddr);
 
             int idx;
+
             if (g.findLast)
-                idx = io.isLeaf()? cnt - 1: -cnt - 1; //(-cnt - 1) mimics not_found result of findInsertionPoint
-                //in case of cnt = 0 we end up in 'not found' branch below with idx being 0 after fix() adjustment
+                idx = io.isLeaf() ? cnt - 1 : -cnt - 1; // (-cnt - 1) mimics not_found result of findInsertionPoint
+                // in case of cnt = 0 we end up in 'not found' branch below with idx being 0 after fix() adjustment
             else
-                idx = findInsertionPoint(io, pageAddr, 0, cnt, g.row, g.shift);
+                idx = findInsertionPoint(lvl, io, pageAddr, 0, cnt, g.row, g.shift);
 
             boolean found = idx >= 0;
 
@@ -338,7 +339,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert p.btmLvl == 0 : "split is impossible with replace";
 
             final int cnt = io.getCount(pageAddr);
-            final int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
+            final int idx = findInsertionPoint(lvl, io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx < 0) // Not found, split or merge happened.
                 return RETRY;
@@ -399,7 +400,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return RETRY;
 
             int cnt = io.getCount(pageAddr);
-            int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
+            int idx = findInsertionPoint(lvl, io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx >= 0) // We do not support concurrent put of the same key.
                 throw new IllegalStateException("Duplicate row in index.");
@@ -451,7 +452,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             assert cnt <= Short.MAX_VALUE: cnt;
 
-            int idx = findInsertionPoint(io, leafAddr, 0, cnt, r.row, 0);
+            int idx = findInsertionPoint(lvl, io, leafAddr, 0, cnt, r.row, 0);
 
             if (idx < 0)
                 return RETRY; // We've found exact match on search but now it's gone.
@@ -1205,7 +1206,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             validateDownPages(rootPageId, 0L, rootLvl);
 
-            validateDownKeys(rootPageId, null);
+            validateDownKeys(rootPageId, null, rootLvl);
         }
         finally {
             releasePage(metaPageId, metaPage);
@@ -1217,7 +1218,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @param minRow Minimum row.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
+    private void validateDownKeys(long pageId, L minRow, int lvl) throws IgniteCheckedException {
         long page = acquirePage(pageId);
         try {
             long pageAddr = readLock(pageId, page); // No correctness guaranties.
@@ -1232,7 +1233,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 if (io.isLeaf()) {
                     for (int i = 0; i < cnt; i++) {
-                        if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
+                        if (minRow != null && compare(lvl, io, pageAddr, i, minRow) <= 0)
                             fail("Wrong sort order: " + U.hexLong(pageId) + " , at " + i + " , minRow: " + minRow);
 
                         minRow = io.getLookupRow(this, pageAddr, i);
@@ -1245,20 +1246,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 for (int i = 0; i < cnt; i++) {
                     L row = io.getLookupRow(this, pageAddr, i);
 
-                    if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
+                    if (minRow != null && compare(lvl, io, pageAddr, i, minRow) <= 0)
                         fail("Min row violated: " + row + " , minRow: " + minRow);
 
                     long leftId = inner(io).getLeft(pageAddr, i);
 
                     L leafRow = getGreatestRowInSubTree(leftId);
 
-                    int cmp = compare(io, pageAddr, i, leafRow);
+                    int cmp = compare(lvl, io, pageAddr, i, leafRow);
 
                     if (cmp < 0 || (cmp != 0 && canGetRowFromInner))
                         fail("Wrong inner row: " + U.hexLong(pageId) + " , at: " + i + " , leaf:  " + leafRow +
                             " , inner: " + row);
 
-                    validateDownKeys(leftId, minRow);
+                    validateDownKeys(leftId, minRow, lvl - 1);
 
                     minRow = row;
                 }
@@ -1266,7 +1267,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Need to handle the rightmost child subtree separately or handle empty routing page.
                 long rightId = inner(io).getLeft(pageAddr, cnt); // The same as getRight(cnt - 1)
 
-                validateDownKeys(rightId, minRow);
+                validateDownKeys(rightId, minRow, lvl - 1);
             }
             finally {
                 readUnlock(pageId, page, pageAddr);
@@ -2694,7 +2695,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 assert fwdPageAddr != 0L;
 
                 // TODO GG-11640 log a correct forward page record.
-                Boolean fwdPageWalPlc = Boolean.TRUE;
+                final Boolean fwdPageWalPlc = Boolean.TRUE;
 
                 try {
                     boolean midShift = splitPage(pageId, page, pageAddr, io, fwdId, fwdPageAddr, idx);
@@ -2742,7 +2743,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             assert newRootAddr != 0L;
 
                             // Never write full new root page, because it is known to be new.
-                            Boolean newRootPageWalPlc = Boolean.FALSE;
+                            final Boolean newRootPageWalPlc = Boolean.FALSE;
 
                             try {
                                 boolean needWal = needWalDeltaRecord(newRootId, newRootPage, newRootPageWalPlc);
@@ -3650,7 +3651,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert tail.type == Tail.EXACT: tail.type;
 
             if (tail.idx == Short.MIN_VALUE) {
-                int idx = findInsertionPoint(tail.io, tail.buf, 0, tail.getCount(), row, 0);
+                int idx = findInsertionPoint(tail.lvl, tail.io, tail.buf, 0, tail.getCount(), row, 0);
 
                 assert checkIndex(idx): idx;
 
@@ -4173,7 +4174,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private byte type;
 
         /** */
-        private final byte lvl;
+        private final int lvl;
 
         /** */
         private short idx = Short.MIN_VALUE;
@@ -4249,7 +4250,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return Insertion point as in {@link Arrays#binarySearch(Object[], Object, Comparator)}.
      * @throws IgniteCheckedException If failed.
      */
-    private int findInsertionPoint(BPlusIO<L> io, long buf, int low, int cnt, L row, int shift)
+    private int findInsertionPoint(int lvl, BPlusIO<L> io, long buf, int low, int cnt, L row, int shift)
         throws IgniteCheckedException {
         assert row != null;
 
@@ -4258,7 +4259,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         while (low <= high) {
             int mid = (low + high) >>> 1;
 
-            int cmp = compare(io, buf, mid, row);
+            int cmp = compare(lvl, io, buf, mid, row);
 
             if (cmp == 0)
                 cmp = -shift; // We need to fix the case when search row matches multiple data rows.
@@ -4329,6 +4330,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     protected abstract int compare(BPlusIO<L> io, long pageAddr, int idx, L row) throws IgniteCheckedException;
 
     /**
+     * @param lvl Level.
+     * @param io IO.
+     * @param pageAddr Page address.
+     * @param idx Index of row in the given buffer.
+     * @param row Lookup row.
+     * @return Comparison result as in {@link Comparator#compare(Object, Object)}.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected int compare(int lvl, BPlusIO<L> io, long pageAddr, int idx, L row) throws IgniteCheckedException {
+        return compare(io, pageAddr, idx, row);
+    }
+
+    /**
      * Get a full detached data row.
      *
      * @param io IO.
@@ -4421,11 +4435,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private int findLowerBound(long pageAddr, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
+            assert io.isLeaf();
+
             // Compare with the first row on the page.
-            int cmp = compare(io, pageAddr, 0, lowerBound);
+            int cmp = compare(0, io, pageAddr, 0, lowerBound);
 
             if (cmp < 0 || (cmp == 0 && lowerShift == 1)) {
-                int idx = findInsertionPoint(io, pageAddr, 0, cnt, lowerBound, lowerShift);
+                int idx = findInsertionPoint(0, io, pageAddr, 0, cnt, lowerBound, lowerShift);
 
                 assert idx < 0;
 
@@ -4444,11 +4460,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private int findUpperBound(long pageAddr, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
+            assert io.isLeaf();
+
             // Compare with the last row on the page.
-            int cmp = compare(io, pageAddr, cnt - 1, upperBound);
+            int cmp = compare(0, io, pageAddr, cnt - 1, upperBound);
 
             if (cmp > 0) {
-                int idx = findInsertionPoint(io, pageAddr, low, cnt, upperBound, 1);
+                int idx = findInsertionPoint(0, io, pageAddr, low, cnt, upperBound, 1);
 
                 assert idx < 0;
 


[20/39] ignite git commit: IGNITE-4947 - Merged Basic and Binary Basic suites. Closes #1991

Posted by sb...@apache.org.
IGNITE-4947 - Merged Basic and Binary Basic suites. Closes #1991


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: af026d1efa482728f1ee78a8d3aa54c2162233f5
Parents: ffb4f3c
Author: Dmitry Pavlov <dp...@gmail.com>
Authored: Thu May 25 10:21:37 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu May 25 10:23:11 2017 +0300

----------------------------------------------------------------------
 ...iteMarshallerCacheClassNameConflictTest.java |  9 +--
 .../ignite/testframework/GridTestUtils.java     |  6 +-
 .../config/GridTestProperties.java              |  3 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java | 13 ++--
 .../testsuites/IgniteBinaryBasicTestSuite.java  | 75 +-------------------
 ...iteBinarySimpleNameMapperBasicTestSuite.java |  3 +-
 6 files changed, 25 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/af026d1e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
index d2a304d..6545a4e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiListener;
 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.config.GridTestProperties;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -108,7 +109,7 @@ public class IgniteMarshallerCacheClassNameConflictTest extends GridCommonAbstra
         final AtomicInteger trickCompilerVar = new AtomicInteger(1);
 
         final Organization aOrg1 = new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA");
-        final OrganizatioN bOrg2 = new OrganizatioN(2, "Apple", "1 Infinite Loop, Cupertino, CA 95014, USA");
+        final Organization_D4pss2X99lE bOrg2 = new Organization_D4pss2X99lE(2, "Apple", "1 Infinite Loop, Cupertino, CA 95014, USA");
 
         exec1.submit(new Runnable() {
             @Override public void run() {
@@ -206,7 +207,7 @@ public class IgniteMarshallerCacheClassNameConflictTest extends GridCommonAbstra
                             rejectObserved = true;
                             if (conflClsName.contains("Organization"))
                                 bbClsRejected = true;
-                            else if (conflClsName.contains("OrganizatioN"))
+                            else if (conflClsName.contains("Organization_D4pss2X99lE"))
                                 aaClsRejected = true;
                         }
                     }
@@ -254,7 +255,7 @@ public class IgniteMarshallerCacheClassNameConflictTest extends GridCommonAbstra
     /**
      * Class name is chosen to be in conflict with other class name this test put to cache.
      */
-    private static class OrganizatioN {
+    private static class Organization_D4pss2X99lE {
         /** */
         private final int id;
 
@@ -269,7 +270,7 @@ public class IgniteMarshallerCacheClassNameConflictTest extends GridCommonAbstra
          * @param name Name.
          * @param addr Address.
          */
-        OrganizatioN(int id, String name, String addr) {
+        Organization_D4pss2X99lE(int id, String name, String addr) {
             this.id = id;
             this.name = name;
             this.addr = addr;

http://git-wip-us.apache.org/repos/asf/ignite/blob/af026d1e/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 8559b97..46d97a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -59,6 +59,7 @@ import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.TrustManager;
 import junit.framework.Test;
+import junit.framework.TestCase;
 import junit.framework.TestSuite;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -1848,9 +1849,10 @@ public final class GridTestUtils {
      *
      * @param suite TestSuite where to place the test.
      * @param test Test.
-     * @param ignoredTests Tests to ignore.
+     * @param ignoredTests Tests to ignore. If test contained in the collection it is not included in suite
      */
-    public static void addTestIfNeeded(TestSuite suite, Class test, Set<Class> ignoredTests) {
+    public static void addTestIfNeeded(@NotNull final TestSuite suite, @NotNull final Class<? extends TestCase> test,
+        @Nullable final Collection<Class> ignoredTests) {
         if (ignoredTests != null && ignoredTests.contains(test))
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/af026d1e/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
index 6d1c3e6..ec7e99b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
@@ -29,6 +29,7 @@ import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.log4j.xml.DOMConfigurator;
 import org.jetbrains.annotations.Nullable;
@@ -79,7 +80,7 @@ public final class GridTestProperties {
     /** Binary marshaller compact footers property. */
     public static final String BINARY_COMPACT_FOOTERS = "binary.marshaller.compact.footers";
 
-    /** Enables {@link BinaryBasicNameMapper} with usage of simple (w/o package) name of class. */
+    /** "True value" enables {@link BinaryBasicNameMapper} in {@link BinaryTypeConfiguration#getNameMapper()}  */
     public static final String BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER = "binary.marshaller.use.simple.name.mapper";
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/af026d1e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index ee69f24..46b1eb5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -20,8 +20,6 @@ package org.apache.ignite.testsuites;
 import java.util.Set;
 import junit.framework.TestSuite;
 import org.apache.ignite.GridSuppressedExceptionSelfTest;
-import org.apache.ignite.internal.processors.database.MemoryMetricsSelfTest;
-import org.apache.ignite.util.AttributeNodeFilterSelfTest;
 import org.apache.ignite.internal.ClusterGroupHostsSelfTest;
 import org.apache.ignite.internal.ClusterGroupSelfTest;
 import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest;
@@ -41,6 +39,8 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRende
 import org.apache.ignite.internal.processors.cache.GridLocalIgniteSerializationTest;
 import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClassNameConflictTest;
+import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClientRequestsMappingOnMissTest;
 import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheConcurrentReadWriteTest;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessorSelfTest;
 import org.apache.ignite.internal.processors.closure.GridClosureSerializationTest;
@@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.database.BPlusTreeFakeReuseSelfTest
 import org.apache.ignite.internal.processors.database.BPlusTreeReuseSelfTest;
 import org.apache.ignite.internal.processors.database.BPlusTreeSelfTest;
 import org.apache.ignite.internal.processors.database.FreeListImplSelfTest;
+import org.apache.ignite.internal.processors.database.MemoryMetricsSelfTest;
 import org.apache.ignite.internal.processors.database.MetadataStorageSelfTest;
 import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerProcessorValidationSelfTest;
@@ -70,6 +71,8 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.test.ConfigVariationsTestSuiteBuilderTest;
 import org.apache.ignite.testframework.test.ParametersTest;
 import org.apache.ignite.testframework.test.VariationsIteratorTest;
+import org.apache.ignite.util.AttributeNodeFilterSelfTest;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Basic test suite.
@@ -84,11 +87,11 @@ public class IgniteBasicTestSuite extends TestSuite {
     }
 
     /**
-     * @param ignoredTests Tests don't include in the execution.
+     * @param ignoredTests Tests don't include in the execution. Providing null means nothing to exclude.
      * @return Test suite.
      * @throws Exception Thrown in case of the failure.
      */
-    public static TestSuite suite(Set<Class> ignoredTests) throws Exception {
+    public static TestSuite suite(@Nullable final Set<Class> ignoredTests) throws Exception {
         TestSuite suite = new TestSuite("Ignite Basic Test Suite");
 
         suite.addTest(IgniteMarshallerSelfTestSuite.suite(ignoredTests));
@@ -168,6 +171,8 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(FreeListImplSelfTest.class);
         suite.addTestSuite(MemoryMetricsSelfTest.class);
 
+        suite.addTestSuite(IgniteMarshallerCacheClassNameConflictTest.class);
+        suite.addTestSuite(IgniteMarshallerCacheClientRequestsMappingOnMissTest.class);
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/af026d1e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
index 837523d..68c463e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
@@ -17,88 +17,19 @@
 
 package org.apache.ignite.testsuites;
 
-import java.util.HashSet;
-import java.util.Set;
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.ClusterGroupSelfTest;
-import org.apache.ignite.internal.GridReleaseTypeSelfTest;
-import org.apache.ignite.internal.GridVersionSelfTest;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.managers.deployment.GridDeploymentMessageCountSelfTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerEnumSelfTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerNodeFailoverTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerPooledSelfTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerSelfTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerSerialPersistentFieldsSelfTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerTest;
-import org.apache.ignite.internal.marshaller.optimized.OptimizedObjectStreamSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingErrorTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingNearErrorTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingRebalanceErrorTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingTxErrorTest;
-import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest;
-import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClassNameConflictTest;
-import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClientRequestsMappingOnMissTest;
-import org.apache.ignite.internal.util.GridHandleTableSelfTest;
-import org.apache.ignite.internal.util.IgniteUtilsSelfTest;
-import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest;
-import org.apache.ignite.internal.util.nio.GridNioSelfTest;
-import org.apache.ignite.internal.util.nio.GridNioSslSelfTest;
-import org.apache.ignite.marshaller.DynamicProxySerializationMultiJvmSelfTest;
-import org.apache.ignite.marshaller.jdk.GridJdkMarshallerSelfTest;
-import org.apache.ignite.messaging.GridMessagingNoPeerClassLoadingSelfTest;
-import org.apache.ignite.messaging.GridMessagingSelfTest;
-import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**
  * Basic test suite.
+ * May be removed soon as all tests were moved to {@link IgniteBasicTestSuite}
  */
+@Deprecated
 public class IgniteBinaryBasicTestSuite extends TestSuite {
     /**
      * @return Test suite.
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suite() throws Exception {
-        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
-
-        TestSuite suite = new TestSuite("GridGain Binary Basic Test Suite");
-
-        Set<Class> ignoredTests = new HashSet<>();
-
-        // Tests that are not ready to be used with PortableMarshaller
-        ignoredTests.add(GridJdkMarshallerSelfTest.class);
-        ignoredTests.add(OptimizedMarshallerEnumSelfTest.class);
-        ignoredTests.add(OptimizedMarshallerSelfTest.class);
-        ignoredTests.add(OptimizedMarshallerTest.class);
-        ignoredTests.add(OptimizedObjectStreamSelfTest.class);
-        ignoredTests.add(GridUnsafeDataOutputArraySizingSelfTest.class);
-        ignoredTests.add(OptimizedMarshallerNodeFailoverTest.class);
-        ignoredTests.add(OptimizedMarshallerSerialPersistentFieldsSelfTest.class);
-        ignoredTests.add(GridNioSslSelfTest.class);
-        ignoredTests.add(GridNioSelfTest.class);
-        ignoredTests.add(IgniteCacheP2pUnmarshallingErrorTest.class);
-        ignoredTests.add(IgniteCacheP2pUnmarshallingTxErrorTest.class);
-        ignoredTests.add(IgniteCacheP2pUnmarshallingNearErrorTest.class);
-        ignoredTests.add(IgniteCacheP2pUnmarshallingRebalanceErrorTest.class);
-        ignoredTests.add(GridReleaseTypeSelfTest.class);
-        ignoredTests.add(IgniteUtilsSelfTest.class);
-        ignoredTests.add(ClusterGroupSelfTest.class);
-        ignoredTests.add(GridMessagingNoPeerClassLoadingSelfTest.class);
-        ignoredTests.add(GridMessagingSelfTest.class);
-        ignoredTests.add(GridVersionSelfTest.class);
-        ignoredTests.add(GridDeploymentMessageCountSelfTest.class);
-        ignoredTests.add(DynamicProxySerializationMultiJvmSelfTest.class);
-        ignoredTests.add(GridHandleTableSelfTest.class);
-        ignoredTests.add(OptimizedMarshallerPooledSelfTest.class);
-
-        // TODO: check and delete if pass.
-        ignoredTests.add(IgniteDaemonNodeMarshallerCacheTest.class);
-
-        suite.addTest(IgniteBasicTestSuite.suite(ignoredTests));
-
-        suite.addTestSuite(IgniteMarshallerCacheClassNameConflictTest.class);
-        suite.addTestSuite(IgniteMarshallerCacheClientRequestsMappingOnMissTest.class);
-
-        return suite;
+        return new TestSuite("GridGain Binary Basic Test Suite: migrated to Ignite Basic Test Suite");
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/af026d1e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
index 77020bd..318f87e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClassNameConflictTest;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**
@@ -33,6 +34,6 @@ public class IgniteBinarySimpleNameMapperBasicTestSuite extends TestSuite {
         GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
         GridTestProperties.setProperty(GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
 
-        return IgniteBinaryBasicTestSuite.suite();
+        return IgniteBasicTestSuite.suite();
     }
 }


[24/39] ignite git commit: .NET: Remove dead code from old LINQ CompiledQuery

Posted by sb...@apache.org.
.NET: Remove dead code from old LINQ CompiledQuery


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 49fcd2cced1f7b5117b8f1a4fca252b31e2c9b90
Parents: db7d776
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu May 25 18:45:51 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu May 25 18:45:51 2017 +0300

----------------------------------------------------------------------
 .../Impl/CacheFieldsQueryExecutor.cs            | 39 --------------------
 .../Impl/CacheQueryableBase.cs                  | 14 ++-----
 .../Impl/ICacheQueryableInternal.cs             |  6 ---
 3 files changed, 3 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/49fcd2cc/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs
index 27082bd..bc7251b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs
@@ -93,45 +93,6 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /// <summary>
-        /// Compiles the query (old method, does not support some scenarios).
-        /// </summary>
-        public Func<object[], IQueryCursor<T>> CompileQuery<T>(QueryModel queryModel, Delegate queryCaller)
-        {
-            Debug.Assert(queryModel != null);
-            Debug.Assert(queryCaller != null);
-
-            var qryData = GetQueryData(queryModel);
-
-            var qryText = qryData.QueryText;
-
-            var selector = GetResultSelector<T>(queryModel.SelectClause.Selector);
-
-            // Compiled query is a delegate with query parameters
-            // Delegate parameters order and query parameters order may differ
-
-            // These are in order of usage in query
-            var qryOrderParams = qryData.ParameterExpressions.OfType<MemberExpression>()
-                .Select(x => x.Member.Name).ToList();
-
-            // These are in order they come from user
-            var userOrderParams = queryCaller.Method.GetParameters().Select(x => x.Name).ToList();
-
-            if ((qryOrderParams.Count != qryData.Parameters.Count) ||
-                (qryOrderParams.Count != userOrderParams.Count))
-                throw new InvalidOperationException("Error compiling query: all compiled query arguments " +
-                    "should come from enclosing delegate parameters.");
-
-            var indices = qryOrderParams.Select(x => userOrderParams.IndexOf(x)).ToArray();
-
-            // Check if user param order is already correct
-            if (indices.SequenceEqual(Enumerable.Range(0, indices.Length)))
-                return args => _cache.QueryFields(GetFieldsQuery(qryText, args), selector);
-
-            // Return delegate with reorder
-            return args => _cache.QueryFields(GetFieldsQuery(qryText,
-                args.Select((x, i) => args[indices[i]]).ToArray()), selector);
-        }
-        /// <summary>
         /// Compiles the query without regard to number or order of arguments.
         /// </summary>
         public Func<object[], IQueryCursor<T>> CompileQuery<T>(QueryModel queryModel)

http://git-wip-us.apache.org/repos/asf/ignite/blob/49fcd2cc/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
index 21a7850..5702f4f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs
@@ -28,16 +28,16 @@ namespace Apache.Ignite.Linq.Impl
     /// <summary>
     /// Base class for cache queryables.
     /// </summary>
-    internal class CacheQueryableBase<T> : QueryableBase<T>, ICacheQueryableInternal
+    internal abstract class CacheQueryableBase<T> : QueryableBase<T>, ICacheQueryableInternal
     {
         /** <inheritdoc /> */
-        public CacheQueryableBase(IQueryProvider provider) : base(provider)
+        protected CacheQueryableBase(IQueryProvider provider) : base(provider)
         {
             // No-op.
         }
 
         /** <inheritdoc /> */
-        public CacheQueryableBase(IQueryProvider provider, Expression expression) : base(provider, expression)
+        protected CacheQueryableBase(IQueryProvider provider, Expression expression) : base(provider, expression)
         {
             // No-op.
         }
@@ -82,14 +82,6 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /** <inheritdoc /> */
-        public Func<object[], IQueryCursor<TQ>> CompileQuery<TQ>(Delegate queryCaller)
-        {
-            var executor = CacheQueryProvider.Executor;
-
-            return executor.CompileQuery<TQ>(GetQueryModel(), queryCaller);
-        }
-
-        /** <inheritdoc /> */
         public Func<object[], IQueryCursor<TQ>> CompileQuery<TQ>(LambdaExpression queryExpression)
         {
             var executor = CacheQueryProvider.Executor;

http://git-wip-us.apache.org/repos/asf/ignite/blob/49fcd2cc/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs
index ffc81b4..b24bc5c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs
@@ -47,12 +47,6 @@ namespace Apache.Ignite.Linq.Impl
         QueryModel GetQueryModel();
 
         /// <summary>
-        /// Compiles the query (the old way).
-        /// </summary>
-        /// <param name="queryCaller">Caller expression to examine argument order.</param>
-        Func<object[], IQueryCursor<T>> CompileQuery<T>(Delegate queryCaller);
-
-        /// <summary>
         /// Compiles the query.
         /// </summary>
         /// <param name="queryExpression">The query expression.</param>


[27/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-5075-cc' into ignite-5075-cc

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-5075-cc' into ignite-5075-cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: e36163fc4e3abe8eb780bc03877c13265f9adea4
Parents: 8cbcef6 ab5aead
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 25 20:39:58 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 25 20:39:58 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryAcknowledgeBuffer.java  | 120 ++++
 .../CacheContinuousQueryDeployableObject.java   | 110 ++++
 .../continuous/CacheContinuousQueryEntry.java   |  59 +-
 .../CacheContinuousQueryEventBuffer.java        | 271 +++++++--
 .../continuous/CacheContinuousQueryHandler.java | 606 ++++---------------
 .../CacheContinuousQueryHandlerV2.java          |   6 +-
 .../CacheContinuousQueryPartitionRecovery.java  | 267 ++++++++
 .../continuous/GridContinuousProcessor.java     |   7 +-
 ...tinuousQueryAsyncFailoverAtomicSelfTest.java |   1 -
 .../CacheContinuousQueryEventBufferTest.java    |  65 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |  79 ++-
 ...niteCacheContinuousQueryBackupQueueTest.java |  13 +-
 12 files changed, 919 insertions(+), 685 deletions(-)
----------------------------------------------------------------------



[30/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: e3500de9f775b091635b06108efc4937f524b6a1
Parents: 01f45c1
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 11:21:25 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 11:28:35 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  6 +-
 .../GridCachePartitionExchangeManager.java      |  2 +-
 .../dht/GridClientPartitionTopology.java        | 31 ++++---
 .../dht/GridDhtPartitionTopology.java           |  9 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 59 ++++++++-----
 .../GridDhtPartitionsExchangeFuture.java        | 34 ++++++--
 .../CacheContinuousQueryEventBuffer.java        | 91 ++++++++++++--------
 .../continuous/CacheContinuousQueryManager.java |  4 +-
 8 files changed, 151 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/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 80f872c..87fe18e 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
@@ -28,7 +28,6 @@ import javax.cache.Cache;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorResult;
-
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -61,8 +60,8 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConfl
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
 import org.apache.ignite.internal.processors.dr.GridDrType;
-import org.apache.ignite.internal.util.IgniteTree;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
+import org.apache.ignite.internal.util.IgniteTree;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple;
@@ -75,7 +74,6 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED;
@@ -1838,8 +1836,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (updateRes.success())
                 updateMetrics(c.op, metrics);
 
-            lsnrs = cctx.continuousQueries().updateListeners(internal, false);
-
             // Continuous query filter should be perform under lock.
             if (lsnrs != null) {
                 CacheObject evtVal = cctx.unwrapTemporary(updateVal);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 5314088..2eec8f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1312,7 +1312,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         top = cacheCtx.topology();
 
                     if (top != null) {
-                        updated |= top.update(null, entry.getValue(), null) != null;
+                        updated |= top.update(null, entry.getValue()) != null;
 
                         cctx.affinity().checkRebalanceState(top, cacheId);
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 1de64c5..43bc609 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -650,11 +650,29 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public void applyUpdateCounters(Map<Integer, T2<Long, Long>> cntrMap) {
+        assert cntrMap != null;
+
+        lock.writeLock().lock();
+
+        try {
+            for (Map.Entry<Integer, T2<Long, Long>> e : cntrMap.entrySet()) {
+                T2<Long, Long> cntr = this.cntrMap.get(e.getKey());
+
+                if (cntr == null || cntr.get2() < e.getValue().get2())
+                    this.cntrMap.put(e.getKey(), e.getValue());
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
     @Nullable @Override public GridDhtPartitionMap update(
         @Nullable GridDhtPartitionExchangeId exchId,
-        GridDhtPartitionMap parts,
-        Map<Integer, T2<Long, Long>> cntrMap
+        GridDhtPartitionMap parts
     ) {
         if (log.isDebugEnabled())
             log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
@@ -733,15 +751,6 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                 }
             }
 
-            if (cntrMap != null) {
-                for (Map.Entry<Integer, T2<Long, Long>> e : cntrMap.entrySet()) {
-                    T2<Long, Long> cntr = this.cntrMap.get(e.getKey());
-
-                    if (cntr == null || cntr.get2() < e.getValue().get2())
-                        this.cntrMap.put(e.getKey(), e.getValue());
-                }
-            }
-
             consistencyCheck();
 
             if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index f9fd852..ffc1d63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -234,12 +234,15 @@ public interface GridDhtPartitionTopology {
     /**
      * @param exchId Exchange ID.
      * @param parts Partitions.
-     * @param cntrMap Partition update counters.
      * @return Local partition map if there were evictions or {@code null} otherwise.
      */
     @Nullable public GridDhtPartitionMap update(@Nullable GridDhtPartitionExchangeId exchId,
-        GridDhtPartitionMap parts,
-        @Nullable Map<Integer, T2<Long, Long>> cntrMap);
+        GridDhtPartitionMap parts);
+
+    /**
+     * @param cntrMap Counters map.
+     */
+    public void applyUpdateCounters(Map<Integer, T2<Long, Long>> cntrMap);
 
     /**
      * Checks if there is at least one owner for each partition in the cache topology.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/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 8e79eda..7adce6e 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
@@ -1256,11 +1256,45 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public void applyUpdateCounters(Map<Integer, T2<Long, Long>> cntrMap) {
+        assert cntrMap != null;
+
+        lock.writeLock().lock();
+
+        try {
+            if (stopping)
+                return;
+
+            for (Map.Entry<Integer, T2<Long, Long>> e : cntrMap.entrySet()) {
+                T2<Long, Long> cntr = this.cntrMap.get(e.getKey());
+
+                if (cntr == null || cntr.get2() < e.getValue().get2())
+                    this.cntrMap.put(e.getKey(), e.getValue());
+            }
+
+            for (int i = 0; i < locParts.length(); i++) {
+                GridDhtLocalPartition part = locParts.get(i);
+
+                if (part == null)
+                    continue;
+
+                T2<Long, Long> cntr = cntrMap.get(part.id());
+
+                if (cntr != null && cntr.get2() > part.updateCounter())
+                    part.updateCounter(cntr.get2());
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+
+        }
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
     @Nullable @Override public GridDhtPartitionMap update(
         @Nullable GridDhtPartitionExchangeId exchId,
-        GridDhtPartitionMap parts,
-        @Nullable Map<Integer, T2<Long, Long>> cntrMap
+        GridDhtPartitionMap parts
     ) {
         if (log.isDebugEnabled())
             log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
@@ -1279,27 +1313,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (stopping)
                 return null;
 
-            if (cntrMap != null) {
-                for (Map.Entry<Integer, T2<Long, Long>> e : cntrMap.entrySet()) {
-                    T2<Long, Long> cntr = this.cntrMap.get(e.getKey());
-
-                    if (cntr == null || cntr.get2() < e.getValue().get2())
-                        this.cntrMap.put(e.getKey(), e.getValue());
-                }
-
-                for (int i = 0; i < locParts.length(); i++) {
-                    GridDhtLocalPartition part = locParts.get(i);
-
-                    if (part == null)
-                        continue;
-
-                    T2<Long, Long> cntr = cntrMap.get(part.id());
-
-                    if (cntr != null && cntr.get2() > part.updateCounter())
-                        part.updateCounter(cntr.get2());
-                }
-            }
-
             if (lastExchangeId != null && exchId != null && lastExchangeId.compareTo(exchId) > 0) {
                 if (log.isDebugEnabled())
                     log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/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 8b8c87c..dfea951 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
@@ -34,7 +34,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.IgniteNeedReconnectException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.PartitionLossPolicy;
@@ -47,18 +46,19 @@ 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.IgniteNeedReconnectException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.pagemem.snapshot.StartFullSnapshotAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
 import org.apache.ignite.internal.processors.cache.CacheInvalidStateException;
+import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask;
 import org.apache.ignite.internal.processors.cache.ClusterState;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
-import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -1098,6 +1098,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         }
     }
 
+    /**
+     * @return {@code True} if exchange triggered by server node join or fail.
+     */
     private boolean serverNotDiscoveryEvent() {
         return discoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT && !CU.clientNode(discoEvt.eventNode());
     }
@@ -1108,10 +1111,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         if (realExchange && !cctx.kernalContext().clientNode() && (serverNotDiscoveryEvent() || affChangeMsg != null)) {
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                if (cacheCtx.isLocal())
+                if (!cacheCtx.affinityNode() || cacheCtx.isLocal())
                     continue;
 
-                cacheCtx.continuousQueries().beforeExchange(exchId.topologyVersion());
+                cacheCtx.continuousQueries().flushBackupQueue(exchId.topologyVersion());
             }
        }
 
@@ -1564,6 +1567,25 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 }
             }
 
+            for (GridDhtPartitionsAbstractMessage msg : msgs.values()) {
+                if (msg instanceof GridDhtPartitionsSingleMessage) {
+                    GridDhtPartitionsSingleMessage msg0 = (GridDhtPartitionsSingleMessage)msg;
+
+                    for (Map.Entry<Integer, GridDhtPartitionMap> entry : msg0.partitions().entrySet()) {
+                        Integer cacheId = entry.getKey();
+                        GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
+
+                        GridDhtPartitionTopology top = cacheCtx != null ? cacheCtx.topology() :
+                            cctx.exchange().clientTopology(cacheId, this);
+
+                        Map<Integer, T2<Long, Long>> cntrs = msg0.partitionUpdateCounters(cacheId);
+
+                        if (cntrs != null)
+                            top.applyUpdateCounters(cntrs);
+                    }
+                }
+            }
+
             if (discoEvt.type() == EVT_NODE_JOINED) {
                 if (cctx.kernalContext().state().active())
                     assignPartitionsStates();
@@ -1795,7 +1817,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             GridDhtPartitionTopology top = cacheCtx != null ? cacheCtx.topology() :
                 cctx.exchange().clientTopology(cacheId, this);
 
-            top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId));
+            top.update(exchId, entry.getValue());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index c59b851..a072240 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -112,8 +112,9 @@ public class CacheContinuousQueryEventBuffer {
                 ret.addAll(pending.values());
             }
 
-            if (curBatch.compareAndSet(batch, null))
-                break;
+            break;
+//            if (curBatch.compareAndSet(batch, null))
+//                break;
         }
 
         return ret;
@@ -235,10 +236,14 @@ public class CacheContinuousQueryEventBuffer {
             for (Map.Entry<Long, CacheContinuousQueryEntry> p : pending.headMap(batch.endCntr, true).entrySet()) {
                 long cntr = p.getKey();
 
-                assert cntr >= batch.startCntr && cntr <= batch.endCntr : cntr;
+                assert cntr <= batch.endCntr;
 
-                if (pending.remove(p.getKey()) != null)
-                    res = batch.processEntry0(res, p.getKey(), p.getValue(), backup);
+                if (pending.remove(p.getKey()) != null) {
+                    if (cntr < batch.startCntr)
+                        res = addResult(res, p.getValue(), backup);
+                    else
+                        res = batch.processEntry0(res, p.getKey(), p.getValue(), backup);
+                }
             }
         }
 
@@ -246,6 +251,43 @@ public class CacheContinuousQueryEventBuffer {
     }
 
     /**
+     * @param res Current result.
+     * @param entry Entry to add.
+     * @param backup Backup entry flag.
+     * @return Updated result.
+     */
+    @Nullable private Object addResult(@Nullable Object res, CacheContinuousQueryEntry entry, boolean backup) {
+        if (res == null) {
+            if (backup)
+                backupQ.add(entry);
+            else
+                res = entry;
+        }
+        else {
+            assert !backup;
+
+            List<CacheContinuousQueryEntry> resList;
+
+            if (res instanceof CacheContinuousQueryEntry) {
+                resList = new ArrayList<>();
+
+                resList.add((CacheContinuousQueryEntry)res);
+            }
+            else {
+                assert res instanceof List : res;
+
+                resList = (List<CacheContinuousQueryEntry>)res;
+            }
+
+            resList.add(entry);
+
+            res = resList;
+        }
+
+        return res;
+    }
+
+    /**
      *
      */
     private class Batch {
@@ -313,7 +355,15 @@ public class CacheContinuousQueryEventBuffer {
                     if (e.isFiltered())
                         filtered++;
                     else {
-                        flushEntry = e;
+                        flushEntry = new CacheContinuousQueryEntry(e.cacheId(),
+                            e.eventType(),
+                            e.key(),
+                            e.value(),
+                            e.oldValue(),
+                            e.isKeepBinary(),
+                            e.partition(),
+                            e.updateCounter(),
+                            e.topologyVersion());
 
                         flushEntry.filteredCount(filtered);
 
@@ -338,8 +388,6 @@ public class CacheContinuousQueryEventBuffer {
                 res.add(filteredEntry(cntr - 1, filtered - 1));
             }
 
-            entries = null;
-
             return res;
         }
 
@@ -399,32 +447,7 @@ public class CacheContinuousQueryEventBuffer {
 
                                 filtered = 0;
 
-                                if (res == null) {
-                                    if (backup)
-                                        backupQ.add(entry0);
-                                    else
-                                        res = entry0;
-                                }
-                                else {
-                                    assert !backup;
-
-                                    List<CacheContinuousQueryEntry> resList;
-
-                                    if (res instanceof CacheContinuousQueryEntry) {
-                                        resList = new ArrayList<>();
-
-                                        resList.add((CacheContinuousQueryEntry)res);
-                                    }
-                                    else {
-                                        assert res instanceof List : res;
-
-                                        resList = (List<CacheContinuousQueryEntry>)res;
-                                    }
-
-                                    resList.add(entry0);
-
-                                    res = resList;
-                                }
+                                res = addResult(res, entry0, backup);
                             }
                             else
                                 filtered++;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e3500de9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index acf351f..7cbb1e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -568,9 +568,9 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
     }
 
     /**
-     * @param topVer Topology version.
+     * @param topVer Finished exchange topology version.
      */
-    public void beforeExchange(AffinityTopologyVersion topVer) {
+    public void flushBackupQueue(AffinityTopologyVersion topVer) {
         for (CacheContinuousQueryListener lsnr : lsnrs.values())
             lsnr.flushBackupQueue(cctx.kernalContext(), topVer);
     }


[11/39] ignite git commit: Minor - corrected TODO

Posted by sb...@apache.org.
Minor - corrected TODO


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: a3908bdab64be29cfbf98aac8e1f74920d22d292
Parents: ff81389
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed May 24 10:16:53 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed May 24 10:17:24 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheMapEntry.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a3908bda/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 edf90d0..6b7d6f5 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
@@ -3405,7 +3405,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     if (obsoleteVersionExtras() != null)
                         return true;
 
-                    // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost.
+                    // TODO IGNITE-5286: need keep removed entries in heap map, otherwise removes can be lost.
                     if (cctx.deferredDelete() && deletedUnlocked())
                         return false;
 


[38/39] ignite git commit: Merge branch 'ignite-5075-cc' into ignite-5075-cc-debug

Posted by sb...@apache.org.
Merge branch 'ignite-5075-cc' into ignite-5075-cc-debug

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 1f3ceeb69f074fd1956556951cd1dabd4be286b2
Parents: 7e56f3a
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 16:46:00 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 16:46:00 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryEventBuffer.java        | 33 +++++++++++---------
 ...ContinuousQueryFailoverAbstractSelfTest.java | 10 ++++++
 2 files changed, 28 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3ceeb6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index d0950ef..e66918c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -166,28 +166,31 @@ public class CacheContinuousQueryEventBuffer {
             if (batch == null || cntr < batch.startCntr) {
                 if (backup)
                     backupQ.add(entry);
+                TestDebugLog.addEntryMessage(part,
+                    cntr,
+                    "buffer rcd small start=" + batch.startCntr +
+                        " cntr=" + cntr +
+                        ", backup=" + backup +
+                        " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());
 
-            TestDebugLog.addEntryMessage(part,
-                cntr,
-                "buffer rcd small start=" + batch.startCntr +
-                    " cntr=" + cntr +
-                    ", backup=" + backup +
-                    " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());return entry;
-        }
+                return entry;
+            }
 
             if (cntr <= batch.endCntr) {
                 res = batch.processEntry0(null, cntr, entry, backup);
 
-        if (
-            res = = RETRY)
+                if (res == RETRY)
                     continue;
             }
-        else{
-            TestDebugLog.addEntryMessage(part,
-                cntr,
-                "buffer add pending start=" + batch.startCntr +
-                    " cntr=" + cntr +
-                    " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());pending.put(cntr, entry);}
+            else {
+                TestDebugLog.addEntryMessage(part,
+                    cntr,
+                    "buffer add pending start=" + batch.startCntr +
+                        " cntr=" + cntr +
+                        " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());
+
+                pending.put(cntr, entry);
+            }
 
             break;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3ceeb6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 3e13359..e63e9bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -880,6 +880,8 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 filtered = !filtered;
             }
 
+            TestDebugLog.addMessage("Stop node " + i);
+
             stopGrid(i);
 
             boolean check = GridTestUtils.waitForCondition(new PAX() {
@@ -895,6 +897,14 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
                 log.info("Missed events for keys: " + keys0);
 
+                Integer k = keys0.iterator().next();
+
+                TestDebugLog.addEntryMessage(ignite(4).affinity(DEFAULT_CACHE_NAME).partition(k), null, "missed event");
+
+                TestDebugLog.printKeyMessages(true, ignite(4).affinity(DEFAULT_CACHE_NAME).partition(k));
+
+                System.exit(2);
+
                 fail("Failed to wait for notifications [exp=" + keys.size() + ", left=" + keys0.size() + ']');
             }
 


[16/39] ignite git commit: IGNITE-5284: Splitted IgniteH2Indexing into several classes. This closes #1999.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/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 d3ee6ff..8e6eeba 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
@@ -17,35 +17,22 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.math.BigDecimal;
 import java.sql.Connection;
-import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.sql.Time;
-import java.sql.Timestamp;
 import java.sql.Types;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -60,7 +47,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.cache.query.QueryCursor;
@@ -97,9 +83,7 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryIndexing;
-import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
@@ -115,16 +99,11 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2InnerIO;
 import org.apache.ignite.internal.processors.query.h2.database.io.H2LeafIO;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2SystemIndexFactory;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
-import org.apache.ignite.internal.processors.query.h2.opt.GridLuceneIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
@@ -135,16 +114,12 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.lang.GridPlainRunnable;
 import org.apache.ignite.internal.util.lang.IgniteInClosure2X;
-import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
-import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
-import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiClosure;
@@ -159,49 +134,18 @@ import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
-import org.h2.api.TableEngine;
-import org.h2.command.CommandInterface;
 import org.h2.command.Prepared;
-import org.h2.command.ddl.CreateTableData;
 import org.h2.command.dml.Insert;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Cursor;
 import org.h2.index.Index;
-import org.h2.jdbc.JdbcConnection;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.jdbc.JdbcStatement;
-import org.h2.message.DbException;
-import org.h2.mvstore.cache.CacheLongKeyLIRS;
-import org.h2.result.SearchRow;
-import org.h2.result.SimpleRow;
-import org.h2.result.SortOrder;
 import org.h2.server.web.WebServer;
-import org.h2.table.Column;
 import org.h2.table.IndexColumn;
-import org.h2.table.TableBase;
 import org.h2.tools.Server;
 import org.h2.util.JdbcUtils;
-import org.h2.value.DataType;
-import org.h2.value.Value;
-import org.h2.value.ValueArray;
-import org.h2.value.ValueBoolean;
-import org.h2.value.ValueByte;
-import org.h2.value.ValueBytes;
-import org.h2.value.ValueDate;
-import org.h2.value.ValueDecimal;
-import org.h2.value.ValueDouble;
-import org.h2.value.ValueFloat;
-import org.h2.value.ValueGeometry;
-import org.h2.value.ValueInt;
-import org.h2.value.ValueJavaObject;
-import org.h2.value.ValueLong;
-import org.h2.value.ValueNull;
-import org.h2.value.ValueShort;
-import org.h2.value.ValueString;
-import org.h2.value.ValueTime;
-import org.h2.value.ValueTimestamp;
-import org.h2.value.ValueUuid;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -219,10 +163,6 @@ import static org.apache.ignite.internal.processors.query.QueryUtils.VAL_FIELD_N
 import static org.apache.ignite.internal.processors.query.QueryUtils.VER_FIELD_NAME;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.distributedJoinMode;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.DEFAULT_COLUMNS_COUNT;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VAL_COL;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VER_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.LOCAL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
 
@@ -244,11 +184,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         PageIO.registerH2(H2InnerIO.VERSIONS, H2LeafIO.VERSIONS);
         H2ExtrasInnerIO.register();
         H2ExtrasLeafIO.register();
-    }
 
-    /** Spatial index class name. */
-    private static final String SPATIAL_IDX_CLS =
-        "org.apache.ignite.internal.processors.query.h2.opt.GridH2SpatialIndex";
+        // Initialize system properties for H2.
+        System.setProperty("h2.objectCache", "false");
+        System.setProperty("h2.serializeJavaObject", "false");
+        System.setProperty("h2.objectCacheMaxPerElementSize", "0"); // Avoid ValueJavaObject caching.
+    }
 
     /** Default DB options. */
     private static final String DB_OPTIONS = ";LOCK_MODE=3;MULTI_THREADED=1;DB_CLOSE_ON_EXIT=FALSE" +
@@ -262,7 +203,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** Dummy metadata for update result. */
     public static final List<GridQueryFieldMetadata> UPDATE_RESULT_META = Collections.<GridQueryFieldMetadata>
-        singletonList(new SqlFieldMetadata(null, null, "UPDATED", Long.class.getName()));
+        singletonList(new H2SqlFieldMetadata(null, null, "UPDATED", Long.class.getName()));
 
     /** */
     private static final int PREPARED_STMT_CACHE_SIZE = 256;
@@ -270,15 +211,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** */
     private static final int TWO_STEP_QRY_CACHE_SIZE = 1024;
 
-    /** */
-    private static final Field COMMAND_FIELD;
-
-    /** */
-    private static final char ESC_CH = '\"';
-
-    /** */
-    private static final String ESC_STR = ESC_CH + "" + ESC_CH;
-
     /** The period of clean up the {@link #stmtCache}. */
     private final Long CLEANUP_STMT_CACHE_PERIOD = Long.getLong(IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD, 10_000);
 
@@ -289,25 +221,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** */
     private GridTimeoutProcessor.CancelableTask stmtCacheCleanupTask;
 
-    /*
-     * Command in H2 prepared statement.
-     */
-    static {
-        // Initialize system properties for H2.
-        System.setProperty("h2.objectCache", "false");
-        System.setProperty("h2.serializeJavaObject", "false");
-        System.setProperty("h2.objectCacheMaxPerElementSize", "0"); // Avoid ValueJavaObject caching.
-
-        try {
-            COMMAND_FIELD = JdbcPreparedStatement.class.getDeclaredField("command");
-
-            COMMAND_FIELD.setAccessible(true);
-        }
-        catch (NoSuchFieldException e) {
-            throw new IllegalStateException("Check H2 version in classpath.", e);
-        }
-    }
-
     /** Logger. */
     @LoggerResource
     private IgniteLogger log;
@@ -319,7 +232,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private Marshaller marshaller;
 
     /** Collection of schemaNames and registered tables. */
-    private final ConcurrentMap<String, Schema> schemas = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<String, H2Schema> schemas = new ConcurrentHashMap8<>();
 
     /** */
     private String dbUrl = "jdbc:h2:mem:";
@@ -346,9 +259,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final ConcurrentMap<Long, GridRunningQueryInfo> runs = new ConcurrentHashMap8<>();
 
     /** */
-    private final ThreadLocal<ConnectionWrapper> connCache = new ThreadLocal<ConnectionWrapper>() {
-        @Nullable @Override public ConnectionWrapper get() {
-            ConnectionWrapper c = super.get();
+    private final ThreadLocal<H2ConnectionWrapper> connCache = new ThreadLocal<H2ConnectionWrapper>() {
+        @Nullable @Override public H2ConnectionWrapper get() {
+            H2ConnectionWrapper c = super.get();
 
             boolean reconnect = true;
 
@@ -371,7 +284,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return c;
         }
 
-        @Nullable @Override protected ConnectionWrapper initialValue() {
+        @Nullable @Override protected H2ConnectionWrapper initialValue() {
             Connection c;
 
             try {
@@ -383,7 +296,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             conns.add(c);
 
-            return new ConnectionWrapper(c);
+            return new H2ConnectionWrapper(c);
         }
     };
 
@@ -400,10 +313,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final ConcurrentMap<QueryTable, GridH2Table> dataTables = new ConcurrentHashMap8<>();
 
     /** Statement cache. */
-    private final ConcurrentHashMap<Thread, StatementCache> stmtCache = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<Thread, H2StatementCache> stmtCache = new ConcurrentHashMap<>();
 
     /** */
-    private final GridBoundedConcurrentLinkedHashMap<TwoStepCachedQueryKey, TwoStepCachedQuery> twoStepCache =
+    private final GridBoundedConcurrentLinkedHashMap<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery> twoStepCache =
         new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
 
     /** */
@@ -430,8 +343,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Connection.
      */
     public Connection connectionForCache(String cacheName) {
+        return connectionForSchema(schema(cacheName));
+    }
+
+    /**
+     * @param schema Schema.
+     * @return Connection.
+     */
+    public Connection connectionForSchema(String schema) {
         try {
-            return connectionForThread(schema(cacheName));
+            return connectionForThread(schema);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);
@@ -449,10 +370,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (useStmtCache) {
             Thread curThread = Thread.currentThread();
 
-            StatementCache cache = stmtCache.get(curThread);
+            H2StatementCache cache = stmtCache.get(curThread);
 
             if (cache == null) {
-                StatementCache cache0 = new StatementCache(PREPARED_STMT_CACHE_SIZE);
+                H2StatementCache cache0 = new H2StatementCache(PREPARED_STMT_CACHE_SIZE);
 
                 cache = stmtCache.putIfAbsent(curThread, cache0);
 
@@ -518,7 +439,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws IgniteCheckedException In case of error.
      */
     private Connection connectionForThread(@Nullable String schema) throws IgniteCheckedException {
-        ConnectionWrapper c = connCache.get();
+        H2ConnectionWrapper c = connCache.get();
 
         if (c == null)
             throw new IgniteCheckedException("Failed to get DB connection for thread (check log for details).");
@@ -645,7 +566,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridCacheVersion ver,
         long expirationTime,
         long link) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
+        H2TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl == null)
             return; // Type was rejected.
@@ -653,39 +574,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (expirationTime == 0)
             expirationTime = Long.MAX_VALUE;
 
-        tbl.tbl.update(k, partId, v, ver, expirationTime, false, link);
+        tbl.table().update(k, partId, v, ver, expirationTime, false, link);
 
-        if (tbl.luceneIdx != null)
-            tbl.luceneIdx.store(k, v, ver, expirationTime);
-    }
-
-    /**
-     * @param o Object.
-     * @return {@code true} If it is a binary object.
-     */
-    private boolean isBinary(CacheObject o) {
-        if (ctx == null)
-            return false;
-
-        return ctx.cacheObjects().isBinaryObject(o);
+        if (tbl.luceneIndex() != null)
+            tbl.luceneIndex().store(k, v, ver, expirationTime);
     }
 
     /**
      * @param cacheName Cache name.
      * @return Cache object context.
      */
-    private CacheObjectContext objectContext(String cacheName) {
-        if (ctx == null)
-            return null;
+    public CacheObjectContext objectContext(String cacheName) {
+        GridCacheContext cctx = cacheContext(cacheName);
 
-        return ctx.cache().internalCache(cacheName).context().cacheObjectContext();
+        return cctx != null ? cctx.cacheObjectContext() : null;
     }
 
     /**
      * @param cacheName Cache name.
      * @return Cache object context.
      */
-    private GridCacheContext cacheContext(String cacheName) {
+    public GridCacheContext cacheContext(String cacheName) {
         if (ctx == null)
             return null;
 
@@ -702,14 +611,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']');
 
-        TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
+        H2TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
 
         if (tbl == null)
             return;
 
-        if (tbl.tbl.update(key, partId, val, ver, 0, true, 0)) {
-            if (tbl.luceneIdx != null)
-                tbl.luceneIdx.remove(key);
+        if (tbl.table().update(key, partId, val, ver, 0, true, 0)) {
+            if (tbl.luceneIndex() != null)
+                tbl.luceneIndex().remove(key);
         }
     }
 
@@ -719,7 +628,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param tbl Table to unregister.
      * @throws IgniteCheckedException If failed to unregister.
      */
-    private void removeTable(TableDescriptor tbl) throws IgniteCheckedException {
+    private void removeTable(H2TableDescriptor tbl) throws IgniteCheckedException {
         assert tbl != null;
 
         if (log.isDebugEnabled())
@@ -755,7 +664,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         tbl.onDrop();
 
-        tbl.schema.tbls.remove(tbl.typeName());
+        tbl.schema().tables().remove(tbl.typeName());
     }
 
     /**
@@ -766,14 +675,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param h2Idx User index.
      * @throws IgniteCheckedException If failed.
      */
-    private void addInitialUserIndex(String cacheName, TableDescriptor desc, GridH2IndexBase h2Idx)
+    private void addInitialUserIndex(String cacheName, H2TableDescriptor desc, GridH2IndexBase h2Idx)
         throws IgniteCheckedException {
-        GridH2Table h2Tbl = desc.tbl;
+        GridH2Table h2Tbl = desc.table();
 
         h2Tbl.proposeUserIndex(h2Idx);
 
         try {
-            String sql = indexCreateSql(desc.fullTableName(), h2Idx, false, desc.schema.escapeAll());
+            String sql = H2Utils.indexCreateSql(desc.fullTableName(), h2Idx, false, desc.schema().escapeAll());
 
             executeSql(cacheName, sql);
         }
@@ -792,15 +701,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         // Locate table.
         String schemaName = schema(cacheName);
 
-        Schema schema = schemas.get(schemaName);
+        H2Schema schema = schemas.get(schemaName);
 
-        TableDescriptor desc = (schema != null ? schema.tbls.get(tblName) : null);
+        H2TableDescriptor desc = (schema != null ? schema.tables().get(tblName) : null);
 
         if (desc == null)
             throw new IgniteCheckedException("Table not found in internal H2 database [schemaName=" + schemaName +
                 ", tblName=" + tblName + ']');
 
-        GridH2Table h2Tbl = desc.tbl;
+        GridH2Table h2Tbl = desc.table();
 
         // Create index.
         final GridH2IndexBase h2Idx = desc.createUserIndex(idxDesc);
@@ -829,7 +738,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             // At this point index is in consistent state, promote it through H2 SQL statement, so that cached
             // prepared statements are re-built.
-            String sql = indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists, schema.escapeAll());
+            String sql = H2Utils.indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists, schema.escapeAll());
 
             executeSql(cacheName, sql);
         }
@@ -847,9 +756,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         throws IgniteCheckedException{
         String schemaName = schema(cacheName);
 
-        Schema schema = schemas.get(schemaName);
+        H2Schema schema = schemas.get(schemaName);
 
-        String sql = indexDropSql(schemaName, idxName, ifExists, schema.escapeAll());
+        String sql = H2Utils.indexDropSql(schemaName, idxName, ifExists, schema.escapeAll());
 
         executeSql(cacheName, sql);
     }
@@ -875,54 +784,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Generate {@code CREATE INDEX} SQL statement for given params.
-     * @param fullTblName Fully qualified table name.
-     * @param h2Idx H2 index.
-     * @param ifNotExists Quietly skip index creation if it exists.
-     * @return Statement string.
-     */
-    private static String indexCreateSql(String fullTblName, GridH2IndexBase h2Idx, boolean ifNotExists,
-        boolean escapeAll) {
-        boolean spatial = F.eq(SPATIAL_IDX_CLS, h2Idx.getClass().getName());
-
-        GridStringBuilder sb = new SB("CREATE ")
-            .a(spatial ? "SPATIAL " : "")
-            .a("INDEX ")
-            .a(ifNotExists ? "IF NOT EXISTS " : "")
-            .a(escapeName(h2Idx.getName(), escapeAll))
-            .a(" ON ")
-            .a(fullTblName)
-            .a(" (");
-
-        boolean first = true;
-
-        for (IndexColumn col : h2Idx.getIndexColumns()) {
-            if (first)
-                first = false;
-            else
-                sb.a(", ");
-
-            sb.a("\"" + col.columnName + "\"").a(" ").a(col.sortType == SortOrder.ASCENDING ? "ASC" : "DESC");
-        }
-
-        sb.a(')');
-
-        return sb.toString();
-    }
-
-    /**
-     * Generate {@code CREATE INDEX} SQL statement for given params.
-     * @param schemaName <b>Quoted</b> schema name.
-     * @param idxName Index name.
-     * @param ifExists Quietly skip index drop if it exists.
-     * @param escapeAll Escape flag.
-     * @return Statement string.
-     */
-    private static String indexDropSql(String schemaName, String idxName, boolean ifExists, boolean escapeAll) {
-        return "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + schemaName + '.' + escapeName(idxName, escapeAll);
-    }
-
-    /**
      * Create sorted index.
      *
      * @param schema Schema.
@@ -932,7 +793,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param cols Columns.
      * @return Index.
      */
-    private GridH2IndexBase createSortedIndex(Schema schema, String name, GridH2Table tbl, boolean pk,
+    public GridH2IndexBase createSortedIndex(H2Schema schema, String name, GridH2Table tbl, boolean pk,
         List<IndexColumn> cols, int inlineSize) {
         try {
             GridCacheContext cctx = schema.cacheContext();
@@ -949,50 +810,20 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
     }
 
-    /**
-     * Create spatial index.
-     *
-     * @param tbl Table.
-     * @param idxName Index name.
-     * @param cols Columns.
-     */
-    private GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, IndexColumn[] cols
-    ) {
-        try {
-            Class<?> cls = Class.forName(SPATIAL_IDX_CLS);
-
-            Constructor<?> ctor = cls.getConstructor(
-                GridH2Table.class,
-                String.class,
-                Integer.TYPE,
-                IndexColumn[].class);
-
-            if (!ctor.isAccessible())
-                ctor.setAccessible(true);
-
-            final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
-
-            return (GridH2IndexBase)ctor.newInstance(tbl, idxName, segments, cols);
-        }
-        catch (Exception e) {
-            throw new IgniteException("Failed to instantiate: " + SPATIAL_IDX_CLS, e);
-        }
-    }
-
     @SuppressWarnings("unchecked")
     @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(
         String cacheName, String qry, String typeName,
         IndexingQueryFilter filters) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
+        H2TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
-        if (tbl != null && tbl.luceneIdx != null) {
+        if (tbl != null && tbl.luceneIndex() != null) {
             GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, cacheName,
                 U.currentTimeMillis(), null, true);
 
             try {
                 runs.put(run.id(), run);
 
-                return tbl.luceneIdx.query(qry, filters);
+                return tbl.luceneIndex().query(qry, filters);
             }
             finally {
                 runs.remove(run.id());
@@ -1005,7 +836,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public void unregisterType(String cacheName, String typeName)
         throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
+        H2TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl != null)
             removeTable(tbl);
@@ -1028,9 +859,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     public GridQueryFieldsResult queryLocalSqlFields(final String cacheName, final String qry,
         @Nullable final Collection<Object> params, final IndexingQueryFilter filter, boolean enforceJoinOrder,
         final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException {
-        final Connection conn = connectionForCache(cacheName);
+        final String schema = schema(cacheName);
+
+        final Connection conn = connectionForSchema(schema);
 
-        setupConnection(conn, false, enforceJoinOrder);
+        H2Utils.setupConnection(conn, false, enforceJoinOrder);
 
         final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
 
@@ -1045,7 +878,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             fldsQry.setEnforceJoinOrder(enforceJoinOrder);
             fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
 
-            return dmlProc.updateSqlFieldsLocal(cacheName, stmt, fldsQry, filter, cancel);
+            return dmlProc.updateSqlFieldsLocal(schema, stmt, fldsQry, filter, cancel);
         }
         else if (DdlStatementsProcessor.isDdlStatement(p))
             throw new IgniteSQLException("DDL statements are supported for the whole cluster only",
@@ -1054,7 +887,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         List<GridQueryFieldMetadata> meta;
 
         try {
-            meta = meta(stmt.getMetaData());
+            meta = H2Utils.meta(stmt.getMetaData());
         }
         catch (SQLException e) {
             throw new IgniteCheckedException("Cannot prepare query metadata", e);
@@ -1075,9 +908,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 runs.putIfAbsent(run.id(), run);
 
                 try {
-                    ResultSet rs = executeSqlQueryWithTimer(cacheName, stmt, conn, qry, params, timeout, cancel);
+                    ResultSet rs = executeSqlQueryWithTimer(schema, stmt, conn, qry, params, timeout, cancel);
 
-                    return new FieldsIterator(rs);
+                    return new H2FieldsIterator(rs);
                 }
                 finally {
                     GridH2QueryContext.clearThreadLocal();
@@ -1106,58 +939,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param rsMeta Metadata.
-     * @return List of fields metadata.
-     * @throws SQLException If failed.
-     */
-    private static List<GridQueryFieldMetadata> meta(ResultSetMetaData rsMeta) throws SQLException {
-        List<GridQueryFieldMetadata> meta = new ArrayList<>(rsMeta.getColumnCount());
-
-        for (int i = 1; i <= rsMeta.getColumnCount(); i++) {
-            String schemaName = rsMeta.getSchemaName(i);
-            String typeName = rsMeta.getTableName(i);
-            String name = rsMeta.getColumnLabel(i);
-            String type = rsMeta.getColumnClassName(i);
-
-            if (type == null) // Expression always returns NULL.
-                type = Void.class.getName();
-
-            meta.add(new SqlFieldMetadata(schemaName, typeName, name, type));
-        }
-
-        return meta;
-    }
-
-    /**
-     * @param stmt Prepared statement.
-     * @return Command type.
-     */
-    private static int commandType(PreparedStatement stmt) {
-        try {
-            return ((CommandInterface)COMMAND_FIELD.get(stmt)).getCommandType();
-        }
-        catch (IllegalAccessException e) {
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Stores rule for constructing schemaName according to cache configuration.
-     *
-     * @param ccfg Cache configuration.
-     * @return Proper schema name according to ANSI-99 standard.
-     */
-    private static String schemaNameFromCacheConf(CacheConfiguration<?, ?> ccfg) {
-        if (ccfg.getSqlSchema() == null)
-            return escapeName(ccfg.getName(), true);
-
-        if (ccfg.getSqlSchema().charAt(0) == ESC_CH)
-            return ccfg.getSqlSchema();
-
-        return ccfg.isSqlEscapeAll() ? escapeName(ccfg.getSqlSchema(), true) : ccfg.getSqlSchema().toUpperCase();
-    }
-
-    /**
      * Prepares sql statement.
      *
      * @param conn Connection.
@@ -1210,7 +991,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         if (timeoutMillis > 0)
-            session(conn).setQueryTimeout(timeoutMillis);
+            H2Utils.session(conn).setQueryTimeout(timeoutMillis);
 
         try {
             return stmt.executeQuery();
@@ -1224,14 +1005,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         finally {
             if (timeoutMillis > 0)
-                session(conn).setQueryTimeout(0);
+                H2Utils.session(conn).setQueryTimeout(0);
         }
     }
 
     /**
      * Executes sql query and prints warning if query is too slow..
      *
-     * @param cacheName Cache name.
+     * @param schema Schema.
      * @param conn Connection,.
      * @param sql Sql query.
      * @param params Parameters.
@@ -1240,21 +1021,21 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public ResultSet executeSqlQueryWithTimer(String cacheName,
+    public ResultSet executeSqlQueryWithTimer(String schema,
         Connection conn,
         String sql,
         @Nullable Collection<Object> params,
         boolean useStmtCache,
         int timeoutMillis,
         @Nullable GridQueryCancel cancel) throws IgniteCheckedException {
-        return executeSqlQueryWithTimer(cacheName, preparedStatementWithParams(conn, sql, params, useStmtCache),
+        return executeSqlQueryWithTimer(schema, preparedStatementWithParams(conn, sql, params, useStmtCache),
             conn, sql, params, timeoutMillis, cancel);
     }
 
     /**
      * Executes sql query and prints warning if query is too slow.
      *
-     * @param cacheName Cache name.
+     * @param schema Schema.
      * @param stmt Prepared statement for query.
      * @param conn Connection.
      * @param sql Sql query.
@@ -1263,7 +1044,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    private ResultSet executeSqlQueryWithTimer(String cacheName, PreparedStatement stmt,
+    private ResultSet executeSqlQueryWithTimer(String schema, PreparedStatement stmt,
         Connection conn,
         String sql,
         @Nullable Collection<Object> params,
@@ -1276,7 +1057,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             long time = U.currentTimeMillis() - start;
 
-            long longQryExecTimeout = schemas.get(schema(cacheName)).ccfg.getLongQueryWarningTimeout();
+            long longQryExecTimeout = schemas.get(schema).cacheContext().config().getLongQueryWarningTimeout();
 
             if (time > longQryExecTimeout) {
                 String msg = "Query execution is too long (" + time + " ms): " + sql;
@@ -1319,18 +1100,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
     }
 
-    /**
-     * @param conn Connection to use.
-     * @param distributedJoins If distributed joins are enabled.
-     * @param enforceJoinOrder Enforce join order of tables.
-     */
-    public static void setupConnection(Connection conn, boolean distributedJoins, boolean enforceJoinOrder) {
-        Session s = session(conn);
-
-        s.setForceJoinOrder(enforceJoinOrder);
-        s.setJoinBatchEnabled(distributedJoins);
-    }
-
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
         final SqlFieldsQuery qry, final boolean keepBinary, final IndexingQueryFilter filter,
@@ -1369,6 +1138,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(final GridCacheContext<?, ?> cctx,
         final SqlQuery qry, final IndexingQueryFilter filter, final boolean keepBinary) throws IgniteCheckedException {
         if (cctx.config().getQueryParallelism() > 1) {
@@ -1390,7 +1160,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             final GridCloseableIterator<IgniteBiTuple<K, V>> i = queryLocalSql(cacheName, sqlQry, alias,
                 F.asList(params), type, filter, cancel);
 
-            return new QueryCursorImpl<Cache.Entry<K, V>>(new Iterable<Cache.Entry<K, V>>() {
+            return new QueryCursorImpl<>(new Iterable<Cache.Entry<K, V>>() {
                 @Override public Iterator<Cache.Entry<K, V>> iterator() {
                     return new ClIter<Cache.Entry<K, V>>() {
                         @Override public void close() throws Exception {
@@ -1430,10 +1200,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Queried rows.
      * @throws IgniteCheckedException If failed.
      */
+    @SuppressWarnings("unchecked")
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(String cacheName,
         final String qry, String alias, @Nullable final Collection<Object> params, String type,
         final IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException {
-        final TableDescriptor tbl = tableDescriptor(type, cacheName);
+        final H2TableDescriptor tbl = tableDescriptor(type, cacheName);
 
         if (tbl == null)
             throw new IgniteSQLException("Failed to find SQL table for type: " + type,
@@ -1443,7 +1214,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         Connection conn = connectionForThread(tbl.schemaName());
 
-        setupConnection(conn, false, false);
+        H2Utils.setupConnection(conn, false, false);
 
         GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter)
             .distributedJoinMode(OFF));
@@ -1454,9 +1225,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         runs.put(run.id(), run);
 
         try {
-            ResultSet rs = executeSqlQueryWithTimer(cacheName, conn, sql, params, true, 0, cancel);
+            ResultSet rs = executeSqlQueryWithTimer(schema(cacheName), conn, sql, params, true, 0, cancel);
 
-            return new KeyValIterator(rs);
+            return new H2KeyValueIterator(rs);
         }
         finally {
             GridH2QueryContext.clearThreadLocal();
@@ -1485,7 +1256,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     ) {
         return new Iterable<List<?>>() {
             @Override public Iterator<List<?>> iterator() {
-                return rdcQryExec.query(cctx, qry, keepCacheObj, enforceJoinOrder, timeoutMillis, cancel, params, parts);
+                return rdcQryExec.query(cctx, qry, keepCacheObj, enforceJoinOrder, timeoutMillis, cancel, params,
+                    parts);
             }
         };
     }
@@ -1497,7 +1269,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         String type = qry.getType();
         String cacheName = cctx.name();
 
-        TableDescriptor tblDesc = tableDescriptor(type, cacheName);
+        H2TableDescriptor tblDesc = tableDescriptor(type, cacheName);
 
         if (tblDesc == null)
             throw new IgniteSQLException("Failed to find SQL table for type: " + type,
@@ -1555,21 +1327,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         };
     }
 
-    /**
-     * @param c Connection.
-     * @return Session.
-     */
-    public static Session session(Connection c) {
-        return (Session)((JdbcConnection)c).getSession();
-    }
-
     /** {@inheritDoc} */
-    @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        boolean keepBinary, GridQueryCancel cancel) {
+    @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx,
+        SqlFieldsQuery qry, boolean keepBinary, GridQueryCancel cancel) {
         final String cacheName = cctx.name();
         final String sqlQry = qry.getSql();
 
-        Connection c = connectionForCache(cacheName);
+        String schema = schema(cctx.name());
+
+        Connection c = connectionForSchema(schema);
 
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
         final boolean distributedJoins = qry.isDistributedJoins();
@@ -1580,9 +1346,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridCacheTwoStepQuery twoStepQry = null;
         List<GridQueryFieldMetadata> meta;
 
-        final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(cacheName, sqlQry, grpByCollocated,
+        final H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(cacheName, sqlQry, grpByCollocated,
             distributedJoins, enforceJoinOrder, qry.isLocal());
-        TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
+        H2TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
         if (cachedQry != null) {
             twoStepQry = cachedQry.twoStepQry.copy();
@@ -1592,7 +1358,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             final UUID locNodeId = ctx.localNodeId();
 
             // Here we will just parse the statement, no need to optimize it at all.
-            setupConnection(c, /*distributedJoins*/false, /*enforceJoinOrder*/true);
+            H2Utils.setupConnection(c, /*distributedJoins*/false, /*enforceJoinOrder*/true);
 
             GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, 0, PREPARE)
                 .distributedJoinMode(distributedJoinMode));
@@ -1606,7 +1372,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 try {
                     while (true) {
                         try {
-                            // Do not cache this statement because the whole two step query object will be cached later on.
+                            // Do not cache this statement because the whole query object will be cached later on.
                             stmt = prepareStatement(c, sqlQry, false);
 
                             break;
@@ -1655,7 +1421,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 if (twoStepQry == null) {
                     if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                         try {
-                            return dmlProc.updateSqlFieldsDistributed(cctx.name(), stmt, qry, cancel);
+                            return dmlProc.updateSqlFieldsDistributed(schema, stmt, qry, cancel);
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DML statement [stmt=" + sqlQry +
@@ -1702,7 +1468,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 twoStepQry.cacheIds(cacheIds);
                 twoStepQry.local(qry.isLocal());
 
-                meta = meta(stmt.getMetaData());
+                meta = H2Utils.meta(stmt.getMetaData());
             }
             catch (IgniteCheckedException e) {
                 throw new CacheException("Failed to bind parameters: [qry=" + sqlQry + ", params=" +
@@ -1731,7 +1497,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         cursor.fieldsMeta(meta);
 
         if (cachedQry == null && !twoStepQry.explain()) {
-            cachedQry = new TwoStepCachedQuery(meta, twoStepQry.copy());
+            cachedQry = new H2TwoStepCachedQuery(meta, twoStepQry.copy());
             twoStepCache.putIfAbsent(cachedQryKey, cachedQry);
         }
 
@@ -1752,16 +1518,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * @throws IllegalStateException if segmented indices used with non-segmented indices.
      */
-    private void checkCacheIndexSegmentation(List<Integer> caches) {
-        if (caches.isEmpty())
+    private void checkCacheIndexSegmentation(List<Integer> cacheIds) {
+        if (cacheIds.isEmpty())
             return; // Nothing to check
 
         GridCacheSharedContext sharedCtx = ctx.cache().context();
 
         int expectedParallelism = 0;
 
-        for (int i = 0; i < caches.size(); i++) {
-            GridCacheContext cctx = sharedCtx.cacheContext(caches.get(i));
+        for (Integer cacheId : cacheIds) {
+            GridCacheContext cctx = sharedCtx.cacheContext(cacheId);
 
             assert cctx != null;
 
@@ -1770,8 +1536,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             if (expectedParallelism == 0)
                 expectedParallelism = cctx.config().getQueryParallelism();
-            else if (cctx.config().getQueryParallelism() != expectedParallelism)
-                throw new IllegalStateException("Using indexes with different parallelism levels in same query is forbidden.");
+            else if (cctx.config().getQueryParallelism() != expectedParallelism) {
+                throw new IllegalStateException("Using indexes with different parallelism levels in same query is " +
+                    "forbidden.");
+            }
         }
     }
 
@@ -1784,7 +1552,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return Prepared statement.
      * @throws IgniteCheckedException In case of error.
      */
-    private String generateQuery(String qry, String tableAlias, TableDescriptor tbl) throws IgniteCheckedException {
+    private String generateQuery(String qry, String tableAlias, H2TableDescriptor tbl) throws IgniteCheckedException {
         assert tbl != null;
 
         final String qry0 = qry;
@@ -1847,9 +1615,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         String schemaName = schema(cacheName);
 
-        Schema schema = schemas.get(schemaName);
+        H2Schema schema = schemas.get(schemaName);
 
-        TableDescriptor tbl = new TableDescriptor(schema, type);
+        H2TableDescriptor tbl = new H2TableDescriptor(this, schema, type);
 
         try {
             Connection conn = connectionForThread(schemaName);
@@ -1895,68 +1663,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         if (type.keyFieldName() != null && !type.fields().containsKey(type.keyFieldName())) {
-            throw new IgniteCheckedException(
-                    MessageFormat.format("Name ''{0}'' must be amongst fields since it is configured as ''keyFieldName'' [type=" +
-                            type.name() + "]", type.keyFieldName()));
+            throw new IgniteCheckedException(MessageFormat.format("Name ''{0}'' must be amongst fields since it " +
+                "is configured as ''keyFieldName'' [type=" + type.name() + "]", type.keyFieldName()));
         }
 
         if (type.valueFieldName() != null && !type.fields().containsKey(type.valueFieldName())) {
-            throw new IgniteCheckedException(
-                    MessageFormat.format("Name ''{0}'' must be amongst fields since it is configured as ''valueFieldName'' [type=" +
-                            type.name() + "]", type.valueFieldName()));
-        }
-    }
-
-    /**
-     * Returns empty string, if {@code nullableString} is empty.
-     *
-     * @param nullableString String for convertion. Could be null.
-     * @return Non null string. Could be empty.
-     */
-    private static String emptyIfNull(String nullableString) {
-        return nullableString == null ? "" : nullableString;
-    }
-
-    /**
-     * Escapes name to be valid SQL identifier. Currently just replaces '.' and '$' sign with '_'.
-     *
-     * @param name Name.
-     * @param escapeAll Escape flag.
-     * @return Escaped name.
-     */
-    public static String escapeName(String name, boolean escapeAll) {
-        if (name == null) // It is possible only for a cache name.
-            return ESC_STR;
-
-        if (escapeAll)
-            return ESC_CH + name + ESC_CH;
-
-        SB sb = null;
-
-        for (int i = 0; i < name.length(); i++) {
-            char ch = name.charAt(i);
-
-            if (!Character.isLetter(ch) && !Character.isDigit(ch) && ch != '_' &&
-                !(ch == '"' && (i == 0 || i == name.length() - 1)) && ch != '-') {
-                // Class name can also contain '$' or '.' - these should be escaped.
-                assert ch == '$' || ch == '.';
-
-                if (sb == null)
-                    sb = new SB();
-
-                sb.a(name.substring(sb.length(), i));
-
-                // Replace illegal chars with '_'.
-                sb.a('_');
-            }
+            throw new IgniteCheckedException(MessageFormat.format("Name ''{0}'' must be amongst fields since it " +
+                "is configured as ''valueFieldName'' [type=" + type.name() + "]", type.valueFieldName()));
         }
-
-        if (sb == null)
-            return name;
-
-        sb.a(name.substring(sb.length(), name.length()));
-
-        return sb.toString();
     }
 
     /**
@@ -1969,7 +1683,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws SQLException If failed to create db table.
      * @throws IgniteCheckedException If failed.
      */
-    private void createTable(String cacheName, Schema schema, TableDescriptor tbl, Connection conn)
+    private void createTable(String cacheName, H2Schema schema, H2TableDescriptor tbl, Connection conn)
         throws SQLException, IgniteCheckedException {
         assert schema != null;
         assert tbl != null;
@@ -1990,14 +1704,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         sql.a(',').a(VER_FIELD_NAME).a(" OTHER INVISIBLE");
 
         for (Map.Entry<String, Class<?>> e : tbl.type().fields().entrySet())
-            sql.a(',').a(escapeName(e.getKey(), escapeAll)).a(' ').a(dbTypeFromClass(e.getValue()));
+            sql.a(',').a(H2Utils.escapeName(e.getKey(), escapeAll)).a(' ').a(dbTypeFromClass(e.getValue()));
 
         sql.a(')');
 
         if (log.isDebugEnabled())
             log.debug("Creating DB table with SQL: " + sql);
 
-        GridH2RowDescriptor rowDesc = new RowDescriptor(tbl.type(), schema);
+        GridH2RowDescriptor rowDesc = new H2RowDescriptor(this, tbl.type(), schema);
 
         H2RowFactory rowFactory = tbl.rowFactory(rowDesc);
 
@@ -2032,6 +1746,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * @param h2Tbl Remove data table.
+     */
+    public void removeDataTable(GridH2Table h2Tbl) {
+        dataTables.remove(h2Tbl.identifier(), h2Tbl);
+    }
+
+    /**
      * Find table for index.
      *
      * @param schemaName Schema name.
@@ -2058,7 +1779,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @return DB type name.
      */
     private String dbTypeFromClass(Class<?> cls) {
-        return DBTypeEnum.fromClass(cls).dBTypeAsString();
+        return H2DatabaseType.fromClass(cls).dBTypeAsString();
     }
 
     /**
@@ -2068,28 +1789,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param cacheName Cache name.
      * @return Table descriptor.
      */
-    @Nullable private TableDescriptor tableDescriptor(String type, String cacheName) {
-        Schema s = schemas.get(schema(cacheName));
-
-        if (s == null)
-            return null;
-
-        return s.tbls.get(type);
-    }
+    @Nullable private H2TableDescriptor tableDescriptor(String type, String cacheName) {
+        String schemaName = schema(cacheName);
 
-    /**
-     * Gets collection of table for given schema name.
-     *
-     * @param schema Schema name.
-     * @return Collection of table descriptors.
-     */
-    private Collection<TableDescriptor> tables(String schema) {
-        Schema s = schemas.get(schema);
+        H2Schema schema = schemas.get(schemaName);
 
-        if (s == null)
-            return Collections.emptySet();
+        if (schema == null)
+            return null;
 
-        return s.tbls.values();
+        return schema.tables().get(type);
     }
 
     /**
@@ -2098,8 +1806,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param cacheName Cache name. {@code null} would be converted to an empty string.
      * @return Schema name. Should not be null since we should not fail for an invalid cache name.
      */
-    private String schema(String cacheName) {
-        return emptyIfNull(cacheName2schema.get(emptyIfNull(cacheName)));
+    public String schema(String cacheName) {
+        String res = cacheName2schema.get(cacheName);
+
+        if (res == null)
+            res = "";
+
+        return res;
     }
 
     /**
@@ -2108,8 +1821,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private void cleanupStatementCache() {
         long cur = U.currentTimeMillis();
 
-        for (Iterator<Map.Entry<Thread, StatementCache>> it = stmtCache.entrySet().iterator(); it.hasNext(); ) {
-            Map.Entry<Thread, StatementCache> entry = it.next();
+        for (Iterator<Map.Entry<Thread, H2StatementCache>> it = stmtCache.entrySet().iterator(); it.hasNext(); ) {
+            Map.Entry<Thread, H2StatementCache> entry = it.next();
 
             Thread t = entry.getKey();
 
@@ -2123,16 +1836,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     @Override public String cacheName(String schemaName) {
         assert schemaName != null;
 
-        Schema schema = schemas.get(schemaName);
+        H2Schema schema = schemas.get(schemaName);
 
         // For the compatibility with conversion from """" to "" inside h2 lib
         if (schema == null) {
-            assert schemaName.isEmpty() || schemaName.charAt(0) != ESC_CH;
+            assert schemaName.isEmpty() || schemaName.charAt(0) != H2Utils.ESC_CH;
 
-            schema = schemas.get(escapeName(schemaName, true));
+            schema = schemas.get(H2Utils.escapeName(schemaName, true));
         }
 
-        return schema.cacheName;
+        return schema.cacheName();
     }
 
     /**
@@ -2142,22 +1855,23 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param type Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
+    @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
     @Override public void rebuildIndexesFromHash(String cacheName,
         GridQueryTypeDescriptor type) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
+        H2TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
 
         if (tbl == null)
             return;
 
-        assert tbl.tbl != null;
+        assert tbl.table() != null;
 
-        assert tbl.tbl.rebuildFromHashInProgress();
+        assert tbl.table().rebuildFromHashInProgress();
 
-        H2PkHashIndex hashIdx = tbl.pkHashIdx;
+        H2PkHashIndex hashIdx = tbl.primaryKeyHashIndex();
 
         Cursor cursor = hashIdx.find((Session)null, null, null);
 
-        int cacheId = CU.cacheId(tbl.schema.ccfg.getName());
+        int cacheId = CU.cacheId(tbl.schema().cacheName());
 
         GridCacheContext cctx = ctx.cache().context().cacheContext(cacheId);
 
@@ -2173,12 +1887,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     synchronized (entry) {
                         // TODO : How to correctly get current value and link here?
 
-                        GridH2Row row = tbl.tbl.rowDescriptor().createRow(entry.key(), entry.partition(),
+                        GridH2Row row = tbl.table().rowDescriptor().createRow(entry.key(), entry.partition(),
                             dataRow.value(), entry.version(), entry.expireTime());
 
                         row.link(dataRow.link());
 
-                        List<Index> indexes = tbl.tbl.getAllIndexes();
+                        List<Index> indexes = tbl.table().getAllIndexes();
 
                         for (int i = 2; i < indexes.size(); i++) {
                             Index idx = indexes.get(i);
@@ -2197,19 +1911,19 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         }
 
-        tbl.tbl.markRebuildFromHashInProgress(false);
+        tbl.table().markRebuildFromHashInProgress(false);
     }
 
     /** {@inheritDoc} */
     @Override public void markForRebuildFromHash(String cacheName, GridQueryTypeDescriptor type) {
-        TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
+        H2TableDescriptor tbl = tableDescriptor(type.name(), cacheName);
 
         if (tbl == null)
             return;
 
-        assert tbl.tbl != null;
+        assert tbl.table() != null;
 
-        tbl.tbl.markRebuildFromHashInProgress(true);
+        tbl.table().markRebuildFromHashInProgress(true);
     }
 
     /**
@@ -2221,18 +1935,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws IgniteCheckedException If failed or {@code -1} if the type is unknown.
      */
     long size(String cacheName, String typeName) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(typeName, cacheName);
+        H2TableDescriptor tbl = tableDescriptor(typeName, cacheName);
 
         if (tbl == null)
             return -1;
 
         Connection conn = connectionForCache(cacheName);
 
-        setupConnection(conn, false, false);
+        H2Utils.setupConnection(conn, false, false);
 
         try {
-            ResultSet rs = executeSqlQuery(conn, prepareStatement(conn, "SELECT COUNT(*) FROM " + tbl.fullTableName(), false),
-                0, null);
+            ResultSet rs = executeSqlQuery(conn, prepareStatement(conn, "SELECT COUNT(*) FROM " + tbl.fullTableName(),
+                false), 0, null);
 
             if (!rs.next())
                 throw new IllegalStateException();
@@ -2409,6 +2123,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         // Local node goes the last to allow parallel execution.
         if (locNode != null) {
+            assert locNodeHnd != null;
+
             if (specialize != null)
                 msg = specialize.apply(locNode, msg);
 
@@ -2503,7 +2219,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
 //        unregisterMBean(); TODO https://issues.apache.org/jira/browse/IGNITE-2139
         if (ctx != null && !ctx.cache().context().database().persistenceEnabled()) {
-            for (Schema schema : schemas.values())
+            for (H2Schema schema : schemas.values())
                 schema.onDrop();
         }
 
@@ -2534,12 +2250,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public void registerCache(String cacheName, GridCacheContext<?, ?> cctx, CacheConfiguration<?, ?> ccfg)
         throws IgniteCheckedException {
-        String schema = schemaNameFromCacheConf(ccfg);
+        String schema = H2Utils.schemaNameFromCacheConfiguration(ccfg);
 
-        if (schemas.putIfAbsent(schema, new Schema(cacheName, schema, cctx, ccfg)) != null)
+        if (schemas.putIfAbsent(schema, new H2Schema(cacheName, schema, cctx, ccfg)) != null)
             throw new IgniteCheckedException("Cache already registered: " + U.maskName(cacheName));
 
-        cacheName2schema.put(emptyIfNull(cacheName), schema);
+        cacheName2schema.put(cacheName, schema);
 
         createSchema(schema);
 
@@ -2549,10 +2265,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public void unregisterCache(String cacheName) {
         String schema = schema(cacheName);
-        Schema rmv = schemas.remove(schema);
+        H2Schema rmv = schemas.remove(schema);
 
         if (rmv != null) {
-            cacheName2schema.remove(emptyIfNull(rmv.cacheName));
+            cacheName2schema.remove(rmv.cacheName());
             mapQryExec.onCacheStop(cacheName);
             dmlProc.onCacheStop(cacheName);
 
@@ -2562,28 +2278,26 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 dropSchema(schema);
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(cacheName), e);
+                U.error(log, "Failed to drop schema on cache stop (will ignore): " + cacheName, e);
             }
 
-            for (TableDescriptor tblDesc : rmv.tbls.values())
-                for (Index idx : tblDesc.tbl.getIndexes())
+            for (H2TableDescriptor tblDesc : rmv.tables().values())
+                for (Index idx : tblDesc.table().getIndexes())
                     idx.close(null);
 
-            for (Iterator<Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
-                it.hasNext(); ) {
-                Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery> e = it.next();
+            for (Iterator<Map.Entry<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery>> it =
+                twoStepCache.entrySet().iterator(); it.hasNext();) {
+                Map.Entry<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery> e = it.next();
 
-                if (F.eq(e.getKey().cacheName, cacheName))
+                if (F.eq(e.getKey().cacheName(), cacheName))
                     it.remove();
             }
         }
     }
 
     /** {@inheritDoc} */
-    @Override public IndexingQueryFilter backupFilter(
-        @Nullable final AffinityTopologyVersion topVer,
-        @Nullable final int[] parts
-    ) {
+    @Override public IndexingQueryFilter backupFilter(@Nullable final AffinityTopologyVersion topVer,
+        @Nullable final int[] parts) {
         final AffinityTopologyVersion topVer0 = topVer != null ? topVer : AffinityTopologyVersion.NONE;
 
         return new IndexingQueryFilter() {
@@ -2665,172 +2379,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         rdcQryExec.onDisconnected(reconnectFut);
     }
 
-    /**
-     * Key for cached two-step query.
-     */
-    private static final class TwoStepCachedQueryKey {
-        /** */
-        private final String cacheName;
-
-        /** */
-        private final String sql;
-
-        /** */
-        private final boolean grpByCollocated;
-
-        /** */
-        private final boolean distributedJoins;
-
-        /** */
-        private final boolean enforceJoinOrder;
-
-        /** */
-        private final boolean isLocal;
-
-        /**
-         * @param cacheName Cache name.
-         * @param sql Sql.
-         * @param grpByCollocated Collocated GROUP BY.
-         * @param distributedJoins Distributed joins enabled.
-         * @param enforceJoinOrder Enforce join order of tables.
-         * @param isLocal Query is local flag.
-         */
-        private TwoStepCachedQueryKey(String cacheName,
-            String sql,
-            boolean grpByCollocated,
-            boolean distributedJoins,
-            boolean enforceJoinOrder,
-            boolean isLocal) {
-            this.cacheName = cacheName;
-            this.sql = sql;
-            this.grpByCollocated = grpByCollocated;
-            this.distributedJoins = distributedJoins;
-            this.enforceJoinOrder = enforceJoinOrder;
-            this.isLocal = isLocal;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TwoStepCachedQueryKey that = (TwoStepCachedQueryKey)o;
-
-            if (grpByCollocated != that.grpByCollocated)
-                return false;
-
-            if (distributedJoins != that.distributedJoins)
-                return false;
-
-            if (enforceJoinOrder != that.enforceJoinOrder)
-                return false;
-
-            if (cacheName != null ? !cacheName.equals(that.cacheName) : that.cacheName != null)
-                return false;
-
-            return isLocal == that.isLocal && sql.equals(that.sql);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = cacheName != null ? cacheName.hashCode() : 0;
-            res = 31 * res + sql.hashCode();
-            res = 31 * res + (grpByCollocated ? 1 : 0);
-            res = res + (distributedJoins ? 2 : 0);
-            res = res + (enforceJoinOrder ? 4 : 0);
-            res = res + (isLocal ? 8 : 0);
-
-            return res;
-        }
-    }
-
-    /**
-     * Cached two-step query.
-     */
-    private static final class TwoStepCachedQuery {
-        /** */
-        final List<GridQueryFieldMetadata> meta;
-
-        /** */
-        final GridCacheTwoStepQuery twoStepQry;
-
-        /**
-         * @param meta Fields metadata.
-         * @param twoStepQry Query.
-         */
-        public TwoStepCachedQuery(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
-            this.meta = meta;
-            this.twoStepQry = twoStepQry;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TwoStepCachedQuery.class, this);
-        }
-    }
-
-    /**
-     * @param c1 First column.
-     * @param c2 Second column.
-     * @return {@code true} If they are the same.
-     */
-    private static boolean equal(IndexColumn c1, IndexColumn c2) {
-        return c1.column.getColumnId() == c2.column.getColumnId();
-    }
-
-    /**
-     * @param cols Columns list.
-     * @param col Column to find.
-     * @return {@code true} If found.
-     */
-    private static boolean containsColumn(List<IndexColumn> cols, IndexColumn col) {
-        for (int i = cols.size() - 1; i >= 0; i--) {
-            if (equal(cols.get(i), col))
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * Check whether columns list contains key or key alias column.
-     *
-     * @param desc Row descriptor.
-     * @param cols Columns list.
-     * @return Result.
-     */
-    private static boolean containsKeyColumn(GridH2RowDescriptor desc, List<IndexColumn> cols) {
-        for (int i = cols.size() - 1; i >= 0; i--) {
-            if (desc.isKeyColumn(cols.get(i).column.getColumnId()))
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * @param desc Row descriptor.
-     * @param cols Columns list.
-     * @param keyCol Primary key column.
-     * @param affCol Affinity key column.
-     * @return The same list back.
-     */
-    private static List<IndexColumn> treeIndexColumns(GridH2RowDescriptor desc, List<IndexColumn> cols, IndexColumn keyCol, IndexColumn affCol) {
-        assert keyCol != null;
-
-        if (!containsKeyColumn(desc, cols))
-            cols.add(keyCol);
-
-        if (affCol != null && !containsColumn(cols, affCol))
-            cols.add(affCol);
-
-        return cols;
-    }
-
-
     /** {@inheritDoc} */
     @Override public Collection<GridRunningQueryInfo> runningQueries(long duration) {
         Collection<GridRunningQueryInfo> res = new ArrayList<>();
@@ -2862,1153 +2410,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Wrapper to store connection and flag is schema set or not.
+     * Closeable iterator.
      */
-    private static class ConnectionWrapper {
-        /** */
-        private Connection conn;
-
-        /** */
-        private volatile String schema;
-
-        /**
-         * @param conn Connection to use.
-         */
-        ConnectionWrapper(Connection conn) {
-            this.conn = conn;
-        }
-
-        /**
-         * @return Schema name if schema is set, null otherwise.
-         */
-        public String schema() {
-            return schema;
-        }
-
-        /**
-         * @param schema Schema name set on this connection.
-         */
-        public void schema(@Nullable String schema) {
-            this.schema = schema;
-        }
-
-        /**
-         * @return Connection.
-         */
-        public Connection connection() {
-            return conn;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ConnectionWrapper.class, this);
-        }
+    private interface ClIter<X> extends AutoCloseable, Iterator<X> {
+        // No-op.
     }
 
-    /**
-     * Enum that helps to map java types to database types.
-     */
-    private enum DBTypeEnum {
-        /** */
-        INT("INT"),
-
-        /** */
-        BOOL("BOOL"),
-
-        /** */
-        TINYINT("TINYINT"),
-
-        /** */
-        SMALLINT("SMALLINT"),
-
-        /** */
-        BIGINT("BIGINT"),
-
-        /** */
-        DECIMAL("DECIMAL"),
-
-        /** */
-        DOUBLE("DOUBLE"),
-
-        /** */
-        REAL("REAL"),
-
-        /** */
-        TIME("TIME"),
-
-        /** */
-        TIMESTAMP("TIMESTAMP"),
-
-        /** */
-        DATE("DATE"),
-
-        /** */
-        VARCHAR("VARCHAR"),
-
-        /** */
-        CHAR("CHAR"),
-
-        /** */
-        BINARY("BINARY"),
-
-        /** */
-        UUID("UUID"),
-
-        /** */
-        ARRAY("ARRAY"),
-
-        /** */
-        GEOMETRY("GEOMETRY"),
-
-        /** */
-        OTHER("OTHER");
-
-        /** Map of Class to enum. */
-        private static final Map<Class<?>, DBTypeEnum> map = new HashMap<>();
-
-        /**
-         * Initialize map of DB types.
-         */
-        static {
-            map.put(int.class, INT);
-            map.put(Integer.class, INT);
-            map.put(boolean.class, BOOL);
-            map.put(Boolean.class, BOOL);
-            map.put(byte.class, TINYINT);
-            map.put(Byte.class, TINYINT);
-            map.put(short.class, SMALLINT);
-            map.put(Short.class, SMALLINT);
-            map.put(long.class, BIGINT);
-            map.put(Long.class, BIGINT);
-            map.put(BigDecimal.class, DECIMAL);
-            map.put(double.class, DOUBLE);
-            map.put(Double.class, DOUBLE);
-            map.put(float.class, REAL);
-            map.put(Float.class, REAL);
-            map.put(Time.class, TIME);
-            map.put(Timestamp.class, TIMESTAMP);
-            map.put(java.util.Date.class, TIMESTAMP);
-            map.put(java.sql.Date.class, DATE);
-            map.put(String.class, VARCHAR);
-            map.put(UUID.class, UUID);
-            map.put(byte[].class, BINARY);
-        }
-
-        /** */
-        private final String dbType;
-
-        /**
-         * Constructs new instance.
-         *
-         * @param dbType DB type name.
-         */
-        DBTypeEnum(String dbType) {
-            this.dbType = dbType;
-        }
-
-        /**
-         * Resolves enum by class.
-         *
-         * @param cls Class.
-         * @return Enum value.
-         */
-        public static DBTypeEnum fromClass(Class<?> cls) {
-            DBTypeEnum res = map.get(cls);
-
-            if (res != null)
-                return res;
-
-            if (DataType.isGeometryClass(cls))
-                return GEOMETRY;
-
-            return cls.isArray() && !cls.getComponentType().isPrimitive() ? ARRAY : OTHER;
-        }
-
-        /**
-         * Gets DB type name.
-         *
-         * @return DB type name.
-         */
-        public String dBTypeAsString() {
-            return dbType;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(DBTypeEnum.class, this);
-        }
-    }
-
-    /**
-     * Information about table in database.
-     */
-    private class TableDescriptor implements GridH2SystemIndexFactory {
-        /** */
-        private final String fullTblName;
-
-        /** */
-        private final GridQueryTypeDescriptor type;
-
-        /** */
-        private final Schema schema;
-
-        /** */
-        private GridH2Table tbl;
-
-        /** */
-        private GridLuceneIndex luceneIdx;
-
-        /** */
-        private H2PkHashIndex pkHashIdx;
-
-        /**
-         * @param schema Schema.
-         * @param type Type descriptor.
-         */
-        TableDescriptor(Schema schema, GridQueryTypeDescriptor type) {
-            this.type = type;
-            this.schema = schema;
-
-            String tblName = escapeName(type.tableName(), schema.escapeAll());
-
-            fullTblName = schema.schemaName + "." + tblName;
-        }
-
-        /**
-         * @return Schema name.
-         */
-        public String schemaName() {
-            return schema.schemaName;
-        }
-
-        /**
-         * @return Database full table name.
-         */
-        String fullTableName() {
-            return fullTblName;
-        }
-
-        /**
-         * @return type name.
-         */
-        String typeName() {
-            return type.name();
-        }
-
-        /**
-         * @return Type.
-         */
-        GridQueryTypeDescriptor type() {
-            return type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TableDescriptor.class, this);
-        }
-
-        /**
-         * Create H2 row factory.
-         *
-         * @param rowDesc Row descriptor.
-         * @return H2 row factory.
-         */
-        H2RowFactory rowFactory(GridH2RowDescriptor rowDesc) {
-            GridCacheContext cctx = schema.cacheContext();
-
-            if (cctx.affinityNode() && cctx.offheapIndex())
-                return new H2RowFactory(rowDesc, cctx);
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public ArrayList<Index> createSystemIndexes(GridH2Table tbl) {
-            ArrayList<Index> idxs = new ArrayList<>();
-
-            IndexColumn keyCol = tbl.indexColumn(KEY_COL, SortOrder.ASCENDING);
-            IndexColumn affCol = tbl.getAffinityKeyColumn();
-
-            if (affCol != null && equal(affCol, keyCol))
-                affCol = null;
-
-            GridH2RowDescriptor desc = tbl.rowDescriptor();
-
-            Index hashIdx = createHashIndex(
-                schema,
-                tbl,
-                "_key_PK_hash",
-                treeIndexColumns(desc, new ArrayList<IndexColumn>(2), keyCol, affCol)
-            );
-
-            if (hashIdx != null)
-                idxs.add(hashIdx);
-
-            // Add primary key index.
-            Index pkIdx = createSortedIndex(
-                schema,
-                "_key_PK",
-                tbl,
-                true,
-                treeIndexColumns(desc, new ArrayList<IndexColumn>(2), keyCol, affCol),
-                -1
-            );
-
-            idxs.add(pkIdx);
-
-            if (type().valueClass() == String.class) {
-                try {
-                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.cacheName, type);
-                }
-                catch (IgniteCheckedException e1) {
-                    throw new IgniteException(e1);
-                }
-            }
-
-            boolean affIdxFound = false;
-
-            GridQueryIndexDescriptor textIdx = type.textIndex();
-
-            if (textIdx != null) {
-                try {
-                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.cacheName, type);
-                }
-                catch (IgniteCheckedException e1) {
-                    throw new IgniteException(e1);
-                }
-            }
-
-            // Locate index where affinity column is first (if any).
-            if (affCol != null) {
-                for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
-                    if (idxDesc.type() != QueryIndexType.SORTED)
-                        continue;
-
-                    String firstField = idxDesc.fields().iterator().next();
-
-                    String firstFieldName =
-                        schema.escapeAll() ? firstField : escapeName(firstField, false).toUpperCase();
-
-                    Column col = tbl.getColumn(firstFieldName);
-
-                    IndexColumn idxCol = tbl.indexColumn(col.getColumnId(),
-                        idxDesc.descending(firstField) ? SortOrder.DESCENDING : SortOrder.ASCENDING);
-
-                    affIdxFound |= equal(idxCol, affCol);
-                }
-            }
-
-            // Add explicit affinity key index if nothing alike was found.
-            if (affCol != null && !affIdxFound) {
-                idxs.add(createSortedIndex(schema, "AFFINITY_KEY", tbl, false,
-                    treeIndexColumns(desc, new ArrayList<IndexColumn>(2), affCol, keyCol), -1));
-            }
-
-            return idxs;
-        }
-
-        /**
-         * Get collection of user indexes.
-         *
-         * @return User indexes.
-         */
-        public Collection<GridH2IndexBase> createUserIndexes() {
-            assert tbl != null;
-
-            ArrayList<GridH2IndexBase> res = new ArrayList<>();
-
-            for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
-                GridH2IndexBase idx = createUserIndex(idxDesc);
-
-                res.add(idx);
-            }
-
-            return res;
-        }
-
-        /**
-         * Create user index.
-         *
-         * @param idxDesc Index descriptor.
-         * @return Index.
-         */
-        private GridH2IndexBase createUserIndex(GridQueryIndexDescriptor idxDesc) {
-            String name = schema.escapeAll() ? idxDesc.name() : escapeName(idxDesc.name(), false).toUpperCase();
-
-            IndexColumn keyCol = tbl.indexColumn(KEY_COL, SortOrder.ASCENDING);
-            IndexColumn affCol = tbl.getAffinityKeyColumn();
-
-            List<IndexColumn> cols = new ArrayList<>(idxDesc.fields().size() + 2);
-
-            boolean escapeAll = schema.escapeAll();
-
-            for (String field : idxDesc.fields()) {
-                String fieldName = escapeAll ? field : escapeName(field, false).toUpperCase();
-
-                Column col = tbl.getColumn(fieldName);
-
-                cols.add(tbl.indexColumn(col.getColumnId(),
-                    idxDesc.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
-            }
-
-            GridH2RowDescriptor desc = tbl.rowDescriptor();
-            if (idxDesc.type() == QueryIndexType.SORTED) {
-                cols = treeIndexColumns(desc, cols, keyCol, affCol);
-                return createSortedIndex(schema, name, tbl, false, cols, idxDesc.inlineSize());
-            }
-            else if (idxDesc.type() == QueryIndexType.GEOSPATIAL) {
-                return createSpatialIndex(tbl, name, cols.toArray(new IndexColumn[cols.size()]));
-            }
-
-            throw new IllegalStateException("Index type: " + idxDesc.type());
-        }
-
-        /**
-         * Create hash index.
-         *
-         * @param schema Schema.
-         * @param tbl Table.
-         * @param idxName Index name.
-         * @param cols Columns.
-         * @return Index.
-         */
-        private Index createHashIndex(Schema schema, GridH2Table tbl, String idxName, List<IndexColumn> cols) {
-            GridCacheContext cctx = schema.cacheContext();
-
-            if (cctx.affinityNode() && cctx.offheapIndex()) {
-                assert pkHashIdx == null : pkHashIdx;
-
-                pkHashIdx = new H2PkHashIndex(cctx, tbl, idxName, cols);
-
-                return pkHashIdx;
-            }
-
-            return null;
-        }
-
-        /**
-         *
-         */
-        void onDrop() {
-            dataTables.remove(tbl.identifier(), tbl);
-
-            tbl.destroy();
-
-            U.closeQuiet(luceneIdx);
-        }
-    }
-
-    /**
-     * Special field set iterator based on database result set.
-     */
-    public static class FieldsIterator extends GridH2ResultSetIterator<List<?>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * @param data Data.
-         * @throws IgniteCheckedException If failed.
-         */
-        public FieldsIterator(ResultSet data) throws IgniteCheckedException {
-            super(data, false, true);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected List<?> createRow() {
-            ArrayList<Object> res = new ArrayList<>(row.length);
-
-            Collections.addAll(res, row);
-
-            return res;
-        }
-    }
-
-    /**
-     * Special key/value iterator based on database result set.
-     */
-    private static class KeyValIterator<K, V> extends GridH2ResultSetIterator<IgniteBiTuple<K, V>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * @param data Data array.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected KeyValIterator(ResultSet data) throws IgniteCheckedException {
-            super(data, false, true);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override protected IgniteBiTuple<K, V> createRow() {
-            K key = (K)row[0];
-            V val = (V)row[1];
-
-            return new IgniteBiTuple<>(key, val);
-        }
-    }
-
-    /**
-     * Closeable iterator.
-     */
-    private interface ClIter<X> extends AutoCloseable, Iterator<X> {
-        // No-op.
-    }
-
-    /**
-     * Field descriptor.
-     */
-    static class SqlFieldMetadata implements GridQueryFieldMetadata {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Schema name. */
-        private String schemaName;
-
-        /** Type name. */
-        private String typeName;
-
-        /** Name. */
-        private String name;
-
-        /** Type. */
-        private String type;
-
-        /**
-         * Required by {@link Externalizable}.
-         */
-        public SqlFieldMetadata() {
-            // No-op
-        }
-
-        /**
-         * @param schemaName Schema name.
-         * @param typeName Type name.
-         * @param name Name.
-         * @param type Type.
-         */
-        SqlFieldMetadata(@Nullable String schemaName, @Nullable String typeName, String name, String type) {
-            assert name != null && type != null : schemaName + " | " + typeName + " | " + name + " | " + type;
-
-            this.schemaName = schemaName;
-            this.typeName = typeName;
-            this.name = name;
-            this.type = type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String schemaName() {
-            return schemaName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String typeName() {
-            return typeName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String fieldName() {
-            return name;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String fieldTypeName() {
-            return type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, schemaName);
-            U.writeString(out, typeName);
-            U.writeString(out, name);
-            U.writeString(out, type);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            schemaName = U.readString(in);
-            typeName = U.readString(in);
-            name = U.readString(in);
-            type = U.readString(in);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(SqlFieldMetadata.class, this);
-        }
-    }
-
-    /**
-     * Database schema object.
-     */
-    private class Schema {
-        /** */
-        private final String cacheName;
-
-        /** */
-        private final String schemaName;
-
-        /** */
-        private final GridUnsafeMemory offheap = null;
-
-        /** */
-        private final ConcurrentMap<String, TableDescriptor> tbls = new ConcurrentHashMap8<>();
-
-        /** Cache for deserialized offheap rows. */
-        private final CacheLongKeyLIRS<GridH2Row> rowCache;
-
-        /** */
-        private final GridCacheContext<?, ?> cctx;
-
-        /** */
-        private final CacheConfiguration<?, ?> ccfg;
-
-        /**
-         * @param cacheName Cache name.
-         * @param schemaName Schema name.
-         * @param cctx Cache context.
-         * @param ccfg Cache configuration.
-         */
-        private Schema(String cacheName, String schemaName, GridCacheContext<?, ?> cctx,
-            CacheConfiguration<?, ?> ccfg) {
-            this.cacheName = cacheName;
-            this.cctx = cctx;
-            this.schemaName = schemaName;
-            this.ccfg = ccfg;
-
-            rowCache = null;
-        }
-
-        /**
-         * @return Cache context.
-         */
-        public GridCacheContext cacheContext() {
-            return cctx;
-        }
-
-        /**
-         * @param tbl Table descriptor.
-         */
-        public void add(TableDescriptor tbl) {
-            if (tbls.putIfAbsent(tbl.typeName(), tbl) != null)
-                throw new IllegalStateException("Table already registered: " + tbl.fullTableName());
-        }
-
-        /**
-         * @return Escape all.
-         */
-        public boolean escapeAll() {
-            return ccfg.isSqlEscapeAll();
-        }
-
-        /**
-         * Called after the schema was dropped.
-         */
-        public void onDrop() {
-            for (TableDescriptor tblDesc : tbls.values())
-                tblDesc.onDrop();
-        }
-    }
-
-    /**
-     * Row descriptor.
-     */
-    private class RowDescriptor implements GridH2RowDescriptor {
-        /** */
-        private final GridQueryTypeDescriptor type;
-
-        /** */
-        private final String[] fields;
-
-        /** */
-        private final int[] fieldTypes;
-
-        /** */
-        private final int keyType;
-
-        /** */
-        private final int valType;
-
-        /** */
-        private final Schema schema;
-
-        /** */
-        private final GridUnsafeGuard guard;
-
-        /** */
-        private final boolean snapshotableIdx;
-
-        /** */
-        private final GridQueryProperty[] props;
-
-        /** Id of user-defined key column */
-        private final int keyAliasColumnId;
-
-        /** Id of user-defined value column */
-        private final int valueAliasColumnId;
-
-        /**
-         * @param type Type descriptor.
-         * @param schema Schema.
-         */
-        RowDescriptor(GridQueryTypeDescriptor type, Schema schema) {
-            assert type != null;
-            assert schema != null;
-
-            this.type = type;
-            this.schema = schema;
-
-            guard = schema.offheap == null ? null : new GridUnsafeGuard();
-
-            Map<String, Class<?>> allFields = new LinkedHashMap<>();
-
-            allFields.putAll(type.fields());
-
-            fields = allFields.keySet().toArray(new String[allFields.size()]);
-
-            fieldTypes = new int[fields.length];
-
-            Class[] classes = allFields.values().toArray(new Class[fields.length]);
-
-            for (int i = 0; i < fieldTypes.length; i++)
-                fieldTypes[i] = DataType.getTypeFromClass(classes[i]);
-
-            keyType = DataType.getTypeFromClass(type.keyClass());
-            valType = DataType.getTypeFromClass(type.valueClass());
-
-            props = new GridQueryProperty[fields.length];
-
-            for (int i = 0; i < fields.length; i++) {
-                GridQueryProperty p = type.property(fields[i]);
-
-                assert p != null : fields[i];
-
-                props[i] = p;
-            }
-
-            final List<String> fieldsList = Arrays.asList(fields);
-            keyAliasColumnId = (type.keyFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.keyFieldName()) : -1;
-            valueAliasColumnId = (type.valueFieldName() != null) ? DEFAULT_COLUMNS_COUNT + fieldsList.indexOf(type.valueFieldName()) : -1;
-
-            // Index is not snapshotable in db-x.
-            snapshotableIdx = false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteH2Indexing indexing() {
-            return IgniteH2Indexing.this;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridQueryTypeDescriptor type() {
-            return type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridCacheContext<?, ?> context() {
-            return schema.cacheContext();
-        }
-
-        /** {@inheritDoc} */
-        @Override public CacheConfiguration configuration() {
-            return schema.ccfg;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridUnsafeGuard guard() {
-            return guard;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cache(GridH2Row row) {
-            long ptr = row.pointer();
-
-            assert ptr > 0 : ptr;
-
-            schema.rowCache.put(ptr, row);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void uncache(long ptr) {
-            schema.rowCache.remove(ptr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridUnsafeMemory memory() {
-            return schema.offheap;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Value wrap(Object obj, int type) throws IgniteCheckedException {
-            assert obj != null;
-
-            if (obj instanceof CacheObject) { // Handle cache object.
-                CacheObject co = (CacheObject)obj;
-
-                if (type == Value.JAVA_OBJECT)
-                    return new GridH2ValueCacheObject(cacheContext(schema.cacheName), co);
-
-                obj = co.value(objectContext(schema.cacheName), false);
-            }
-
-            switch (type) {
-                case Value.BOOLEAN:
-                    return ValueBoolean.get((Boolean)obj);
-                case Value.BYTE:
-                    return ValueByte.get((Byte)obj);
-                case Value.SHORT:
-                    return ValueShort.get((Short)obj);
-                case Value.INT:
-                    return ValueInt.get((Integer)obj);
-                case Value.FLOAT:
-                    return ValueFloat.get((Float)obj);
-                case Value.LONG:
-                    return ValueLong.get((Long)obj);
-                case Value.DOUBLE:
-                    return ValueDouble.get((Double)obj);
-                case Value.UUID:
-                    UUID uuid = (UUID)obj;
-                    return ValueUuid.get(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits());
-                case Value.DATE:
-                    return ValueDate.get((Date)obj);
-                case Value.TIME:
-                    return ValueTime.get((Time)obj);
-                case Value.TIMESTAMP:
-                    if (obj instanceof java.util.Date && !(obj instanceof Timestamp))
-                        obj = new Timestamp(((java.util.Date)obj).getTime());
-
-                    return ValueTimestamp.get((Timestamp)obj);
-                case Value.DECIMAL:
-                    return ValueDecimal.get((BigDecimal)obj);
-                case Val

<TRUNCATED>

[23/39] ignite git commit: IGNITE-5124 - Memory metrics API Improvements - Fixes #1988.

Posted by sb...@apache.org.
IGNITE-5124 - Memory metrics API Improvements - Fixes #1988.

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

Branch: refs/heads/ignite-5075-cc-debug
Commit: db7d776ad53edee819dcde9ab3abef48219fe514
Parents: e8f5af3
Author: Sergey Chugunov <se...@gmail.com>
Authored: Thu May 25 18:40:00 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu May 25 18:40:00 2017 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java | 18 +++-
 .../MemoryPolicyConfiguration.java              | 96 +++++++++++++++++++-
 .../apache/ignite/internal/IgniteKernal.java    | 12 +++
 .../IgniteCacheDatabaseSharedManager.java       | 40 ++++++++
 .../cache/database/MemoryMetricsImpl.java       | 31 ++++---
 .../cache/MemoryPolicyConfigValidationTest.java | 62 ++++++++++++-
 .../processors/igfs/IgfsIgniteMock.java         |  7 ++
 .../ignite/testframework/junits/IgniteMock.java |  5 +
 .../junits/multijvm/IgniteProcessProxy.java     |  5 +
 .../org/apache/ignite/IgniteSpringBean.java     |  7 ++
 10 files changed, 266 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 671efca..8d5e967 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -27,6 +27,8 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.lang.IgniteProductVersion;
@@ -618,8 +620,22 @@ public interface Ignite extends AutoCloseable {
     /**
      * Returns a collection of {@link MemoryMetrics} that reflects page memory usage on this Apache Ignite node
      * instance.
+     * Returns the collection that contains the latest snapshots for each memory region
+     * configured with {@link MemoryPolicyConfiguration configuration} on this Ignite node instance.
      *
-     * @return Collection of {@link MemoryMetrics}
+     * @return Collection of {@link MemoryMetrics} snapshots.
      */
     public Collection<MemoryMetrics> memoryMetrics();
+
+    /**
+     * Returns the latest {@link MemoryMetrics} snapshot for the memory region of the given name.
+     *
+     * To get the metrics for the default memory region use
+     * {@link MemoryConfiguration#DFLT_MEM_PLC_DEFAULT_NAME} as the name
+     * or a custom name if the default memory region has been renamed.
+     *
+     * @param memPlcName Name of memory region configured with {@link MemoryPolicyConfiguration config}.
+     * @return {@link MemoryMetrics} snapshot or {@code null} if no memory region is configured under specified name.
+     */
+    @Nullable public MemoryMetrics memoryMetrics(String memPlcName);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index 2716d8e..be0877f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -17,7 +17,9 @@
 package org.apache.ignite.configuration;
 
 import java.io.Serializable;
+import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
+import org.apache.ignite.mxbean.MemoryMetricsMXBean;
 
 import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEM_PLC_DEFAULT_NAME;
 
@@ -66,6 +68,12 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** Default metrics enabled flag. */
     public static final boolean DFLT_METRICS_ENABLED = false;
 
+    /** Default amount of sub intervals to calculate {@link MemoryMetrics#getAllocationRate()} metric. */
+    public static final int DFLT_SUB_INTERVALS = 5;
+
+    /** Default length of interval over which {@link MemoryMetrics#getAllocationRate()} metric is calculated. */
+    public static final int DFLT_RATE_TIME_INTERVAL_SEC = 60;
+
     /** Memory policy name. */
     private String name = DFLT_MEM_PLC_DEFAULT_NAME;
 
@@ -90,9 +98,28 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** Minimum number of empty pages in reuse lists. */
     private int emptyPagesPoolSize = 100;
 
-    /** */
+    /**
+     * Flag to enable the memory metrics collection for this memory policy.
+     */
     private boolean metricsEnabled = DFLT_METRICS_ENABLED;
 
+    /** Number of sub-intervals the whole {@link #setRateTimeInterval(int)} will be split into to calculate
+     * {@link MemoryMetrics#getAllocationRate()} and {@link MemoryMetrics#getEvictionRate()} rates (5 by default).
+     * <p>
+     * Setting it to a bigger value will result in more precise calculation and smaller drops of
+     * {@link MemoryMetrics#getAllocationRate()} metric when next sub-interval has to be recycled but introduces bigger
+     * calculation overhead. */
+    private int subIntervals = DFLT_SUB_INTERVALS;
+
+    /**
+     * Time interval for {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
+     * <p>
+     * For instance, after setting the interval to 60 seconds, subsequent calls to {@link MemoryMetrics#getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     */
+    private int rateTimeInterval = DFLT_RATE_TIME_INTERVAL_SEC;
+
     /**
      * Gets memory policy name.
      *
@@ -263,7 +290,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
 
     /**
      * Gets whether memory metrics are enabled by default on node startup. Memory metrics can be enabled and disabled
-     * at runtime via memory metrics MX bean.
+     * at runtime via memory metrics {@link MemoryMetricsMXBean MX bean}.
      *
      * @return Metrics enabled flag.
      */
@@ -273,7 +300,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
 
     /**
      * Sets memory metrics enabled flag. If this flag is {@code true}, metrics will be enabled on node startup.
-     * Memory metrics can be enabled and disabled at runtime via memory metrics MX bean.
+     * Memory metrics can be enabled and disabled at runtime via memory metrics {@link MemoryMetricsMXBean MX bean}.
      *
      * @param metricsEnabled Metrics enabled flag.
      * @return {@code this} for chaining.
@@ -283,4 +310,67 @@ public final class MemoryPolicyConfiguration implements Serializable {
 
         return this;
     }
+
+    /**
+     * Gets time interval for {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
+     * <p>
+     * For instance, after setting the interval to 60 seconds,
+     * subsequent calls to {@link MemoryMetrics#getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     *
+     * @return Time interval over which allocation rate is calculated.
+     */
+    public int getRateTimeInterval() {
+        return rateTimeInterval;
+    }
+
+    /**
+     * Sets time interval for {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
+     * <p>
+     * For instance, after setting the interval to 60 seconds,
+     * subsequent calls to {@link MemoryMetrics#getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     *
+     * @param rateTimeInterval Time interval used for allocation and eviction rates calculations.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setRateTimeInterval(int rateTimeInterval) {
+        this.rateTimeInterval = rateTimeInterval;
+
+        return this;
+    }
+
+    /**
+     * Gets a number of sub-intervals the whole {@link #setRateTimeInterval(int)}
+     * will be split into to calculate {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} rates (5 by default).
+     * <p>
+     * Setting it to a bigger value will result in more precise calculation and smaller drops of
+     * {@link MemoryMetrics#getAllocationRate()} metric when next sub-interval has to be recycled but introduces bigger
+     * calculation overhead.
+     *
+     * @return number of sub intervals.
+     */
+    public int getSubIntervals() {
+        return subIntervals;
+    }
+
+    /**
+     * Sets a number of sub-intervals the whole {@link #setRateTimeInterval(int)} will be split into to calculate
+     * {@link MemoryMetrics#getAllocationRate()} and {@link MemoryMetrics#getEvictionRate()} rates (5 by default).
+     * <p>
+     * Setting it to a bigger value will result in more precise calculation and smaller drops of
+     * {@link MemoryMetrics#getAllocationRate()} metric when next sub-interval has to be recycled but introduces bigger
+     * calculation overhead.
+     *
+     * @param subIntervals A number of sub-intervals.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setSubIntervals(int subIntervals) {
+        this.subIntervals = subIntervals;
+
+        return this;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/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 0f43169..6393e9f 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
@@ -3426,6 +3426,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        guard();
+
+        try {
+            return ctx.cache().context().database().memoryMetrics(memPlcName);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) {
         guard();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
index 17c16b8..e356f35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
@@ -340,6 +340,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
                 checkPolicySize(plcCfg);
 
+                checkMetricsProperties(plcCfg);
+
                 checkPolicyEvictionProperties(plcCfg, memCfg);
             }
         }
@@ -351,6 +353,26 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @param plcCfg Memory policy config.
+     *
+     * @throws IgniteCheckedException if validation of memory metrics properties fails.
+     */
+    private static void checkMetricsProperties(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
+        if (plcCfg.getRateTimeInterval() <= 0)
+            throw new IgniteCheckedException("Rate time interval must be greater than zero " +
+                "(use MemoryPolicyConfiguration.rateTimeInterval property to adjust the interval) " +
+                "[name=" + plcCfg.getName() +
+                ", rateTimeInterval=" + plcCfg.getRateTimeInterval() + "]"
+            );
+        if (plcCfg.getSubIntervals() <= 0)
+            throw new IgniteCheckedException("Sub intervals must be greater than zero " +
+                "(use MemoryPolicyConfiguration.subIntervals property to adjust the sub intervals) " +
+                "[name=" + plcCfg.getName() +
+                ", subIntervals=" + plcCfg.getSubIntervals() + "]"
+            );
+    }
+
+    /**
      * @param sysCacheInitSize System cache initial size.
      * @param sysCacheMaxSize System cache max size.
      *
@@ -541,6 +563,24 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @param memPlcName Name of {@link MemoryPolicy} to obtain {@link MemoryMetrics} for.
+     * @return {@link MemoryMetrics} snapshot for specified {@link MemoryPolicy} or {@code null} if
+     * no {@link MemoryPolicy} is configured for specified name.
+     */
+    @Nullable public MemoryMetrics memoryMetrics(String memPlcName) {
+        if (!F.isEmpty(memMetricsMap)) {
+            MemoryMetrics memMetrics = memMetricsMap.get(memPlcName);
+
+            if (memMetrics == null)
+                return null;
+            else
+                return new MemoryMetricsSnapshot(memMetrics);
+        }
+        else
+            return null;
+    }
+
+    /**
      * @param memPlcName Memory policy name.
      * @return {@link MemoryPolicy} instance associated with a given {@link MemoryPolicyConfiguration}.
      * @throws IgniteCheckedException in case of request for unknown MemoryPolicy.

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
index ee356a1..3e3b690 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
@@ -44,10 +44,10 @@ public class MemoryMetricsImpl implements MemoryMetrics {
     private volatile boolean metricsEnabled;
 
     /** */
-    private volatile int subInts = 5;
+    private volatile int subInts;
 
     /** */
-    private volatile LongAdder8[] allocRateCounters = new LongAdder8[subInts];
+    private volatile LongAdder8[] allocRateCounters;
 
     /** */
     private final AtomicInteger counterIdx = new AtomicInteger(0);
@@ -58,9 +58,8 @@ public class MemoryMetricsImpl implements MemoryMetrics {
     /** */
     private final MemoryPolicyConfiguration memPlcCfg;
 
-    /** Time interval (in seconds) when allocations/evictions are counted to calculate rate.
-     * Default value is 60 seconds. */
-    private volatile int rateTimeInterval = 60;
+    /** Time interval (in seconds) when allocations/evictions are counted to calculate rate. */
+    private volatile int rateTimeInterval;
 
     /**
      * @param memPlcCfg MemoryPolicyConfiguration.
@@ -70,6 +69,12 @@ public class MemoryMetricsImpl implements MemoryMetrics {
 
         metricsEnabled = memPlcCfg.isMetricsEnabled();
 
+        rateTimeInterval = memPlcCfg.getRateTimeInterval();
+
+        subInts = memPlcCfg.getSubIntervals();
+
+        allocRateCounters = new LongAdder8[subInts];
+
         for (int i = 0; i < subInts; i++)
             allocRateCounters[i] = new LongAdder8();
     }
@@ -140,9 +145,6 @@ public class MemoryMetricsImpl implements MemoryMetrics {
      *
      */
     private void updateAllocationRateMetrics() {
-        if (subInts != allocRateCounters.length)
-            return;
-
         long lastUpdT = lastUpdTime.get();
         long currT = IgniteUtils.currentTimeMillis();
 
@@ -154,11 +156,16 @@ public class MemoryMetricsImpl implements MemoryMetrics {
 
         LongAdder8[] rateCntrs = allocRateCounters;
 
+        if (subInts != rateCntrs.length)
+            return;
+
+        int cntrIdx = counterIdx.get();
+
         for (int i = 1; i <= subInts; i++) {
             if (deltaT < subInt(i)) {
                 if (i > 1) {
                     if (!lastUpdTime.compareAndSet(lastUpdT, currT)) {
-                        rateCntrs[counterIdx.get()].increment();
+                        rateCntrs[cntrIdx].increment();
 
                         break;
                     }
@@ -174,7 +181,7 @@ public class MemoryMetricsImpl implements MemoryMetrics {
                     break;
                 }
                 else {
-                    rateCntrs[counterIdx.get()].increment();
+                    rateCntrs[cntrIdx].increment();
 
                     break;
                 }
@@ -182,8 +189,8 @@ public class MemoryMetricsImpl implements MemoryMetrics {
             else if (i == subInts && lastUpdTime.compareAndSet(lastUpdT, currT))
                 resetAll();
 
-            if (currIdx != counterIdx.get()) {
-                rateCntrs[counterIdx.get()].increment();
+            if (currIdx != cntrIdx) {
+                rateCntrs[cntrIdx].increment();
 
                 break;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
index a1a05eb..1d8174b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
@@ -92,6 +92,16 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
 
                 break;
 
+            case LTE_ZERO_RATE_TIME_INTERVAL:
+                plcs = createRateTimeIntervalIsNegative();
+
+                break;
+
+            case LTE_ZERO_SUB_INTERVALS:
+                plcs = createSubIntervalsIsNegative();
+
+                break;
+
             default:
                 fail("Violation type was not configured: " + violationType);
         }
@@ -106,6 +116,30 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     /**
      *
      */
+    private MemoryPolicyConfiguration[] createSubIntervalsIsNegative() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy(VALID_DEFAULT_MEM_PLC_NAME, 100 * 1024 * 1024, 100 * 1024 * 1024);
+        res[0].setSubIntervals(-10);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createRateTimeIntervalIsNegative() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy(VALID_DEFAULT_MEM_PLC_NAME, 100 * 1024 * 1024, 100 * 1024 * 1024);
+        res[0].setRateTimeInterval(-10);
+
+        return res;
+    }
+
+    /**
+     *
+     */
     private MemoryPolicyConfiguration[] createValidUserDefault() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
 
@@ -277,6 +311,24 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testRateTimeIntervalPropertyIsNegative() throws Exception {
+        violationType = ValidationViolationType.LTE_ZERO_RATE_TIME_INTERVAL;
+
+        doTest(violationType);
+    }
+
+    /**
+     *
+     */
+    public void testSubIntervalsPropertyIsNegative() throws Exception {
+        violationType = ValidationViolationType.LTE_ZERO_SUB_INTERVALS;
+
+        doTest(violationType);
+    }
+
+    /**
      * Tries to start ignite node with invalid configuration and checks that corresponding exception is thrown.
      *
      * @param violationType Configuration violation type.
@@ -323,7 +375,15 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
         TOO_SMALL_USER_DEFINED_DFLT_MEM_PLC_SIZE("User-defined default MemoryPolicy size is less than 1MB."),
 
         /** */
-        MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE("MemoryPolicy maxSize must not be smaller than initialSize");
+        MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE("MemoryPolicy maxSize must not be smaller than initialSize"),
+
+        /** Case when rateTimeInterval property of MemoryPolicyConfiguration is less than or equals zero. */
+        LTE_ZERO_RATE_TIME_INTERVAL("Rate time interval must be greater than zero " +
+            "(use MemoryPolicyConfiguration.rateTimeInterval property to adjust the interval)"),
+
+        /** Case when subIntervals property of MemoryPolicyConfiguration is less than or equals zero. */
+        LTE_ZERO_SUB_INTERVALS("Sub intervals must be greater than zero " +
+            "(use MemoryPolicyConfiguration.subIntervals property to adjust the sub intervals)");
 
         /**
          * @param violationMsg Violation message.

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index cc058b1..ed0af3c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -546,6 +546,13 @@ public class IgfsIgniteMock implements IgniteEx {
         return null;
     }
 
+    /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        throwUnsupported();
+
+        return null;
+    }
+
     /**
      * Throw {@link UnsupportedOperationException}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index ea53345..1f95dd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -446,6 +446,11 @@ public class IgniteMock implements Ignite {
         return null;
     }
 
+    /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        return null;
+    }
+
     /**
      * @param staticCfg Configuration.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 34ca22f..f459e57 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -662,6 +662,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void close() throws IgniteException {
         if (locJvmGrid != null) {
             final CountDownLatch rmtNodeStoppedLatch = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index b1994ac..094c2dc 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -279,6 +279,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        checkIgnite();
+
+        return g.memoryMetrics(memPlcName);
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable String name) {
         checkIgnite();
 


[29/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 01f45c1b8e2dcb2a5eea0596a55c434f682cc60b
Parents: e36163f
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 07:01:40 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 07:01:40 2017 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |  16 +-
 .../CacheContinuousQueryEventBuffer.java        | 150 ++++++++++++-------
 .../continuous/CacheContinuousQueryHandler.java |   4 +-
 3 files changed, 107 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/01f45c1b/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 544f847..8b8c87c 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
@@ -789,14 +789,11 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         boolean topChanged = discoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT || affChangeMsg != null;
 
-        //todo check
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
             if (cacheCtx.isLocal() || stopping(cacheCtx.cacheId()))
                 continue;
 
             if (topChanged) {
-                cacheCtx.continuousQueries().beforeExchange(exchId.topologyVersion());
-
                 // Partition release future is done so we can flush the write-behind store.
                 cacheCtx.store().forceFlush();
             }
@@ -1101,10 +1098,23 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         }
     }
 
+    private boolean serverNotDiscoveryEvent() {
+        return discoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT && !CU.clientNode(discoEvt.eventNode());
+    }
+
     /** {@inheritDoc} */
     @Override public boolean onDone(@Nullable AffinityTopologyVersion res, @Nullable Throwable err) {
         boolean realExchange = !dummy && !forcePreload;
 
+        if (realExchange && !cctx.kernalContext().clientNode() && (serverNotDiscoveryEvent() || affChangeMsg != null)) {
+            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                if (cacheCtx.isLocal())
+                    continue;
+
+                cacheCtx.continuousQueries().beforeExchange(exchId.topologyVersion());
+            }
+       }
+
         if (err == null && realExchange) {
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
                 if (cacheCtx.isLocal())

http://git-wip-us.apache.org/repos/asf/ignite/blob/01f45c1b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index fd4029c..c59b851 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -40,6 +40,9 @@ public class CacheContinuousQueryEventBuffer {
         IgniteSystemProperties.getInteger("IGNITE_CONTINUOUS_QUERY_SERVER_BUFFER_SIZE", 5);
 
     /** */
+    private static final Object RETRY = new Object();
+
+    /** */
     protected final int part;
 
     /** */
@@ -75,32 +78,42 @@ public class CacheContinuousQueryEventBuffer {
     /**
      * @return Backup entries.
      */
-    @Nullable Collection<CacheContinuousQueryEntry> resetBackupQueue() {
-        Collection<CacheContinuousQueryEntry> ret;
+    @Nullable Collection<CacheContinuousQueryEntry> flushOnExchange() {
+        Collection<CacheContinuousQueryEntry> ret = null;
 
-        List<CacheContinuousQueryEntry> entries = null;
+        for (;;) {
+            Batch batch = curBatch.get();
 
-        Batch batch = curBatch.get();
+            if (batch != null) {
+                Collection<CacheContinuousQueryEntry> ret0 = batch.flushAndReset();
 
-        if (batch != null)
-            entries = batch.backupFlushEntries();
+                if (ret0 != null) {
+                    if (ret == null)
+                        ret = ret0;
+                    else
+                        ret.addAll(ret0);
+                }
+            }
 
-        if (!backupQ.isEmpty()) {
-            if (entries != null)
-                backupQ.addAll(entries);
+            if (!backupQ.isEmpty()) {
+                if (ret == null)
+                    ret = new ArrayList<>();
 
-            ret = this.backupQ;
+                CacheContinuousQueryEntry e;
 
-            backupQ = new ConcurrentLinkedDeque<>();
-        }
-        else
-            ret = entries;
+                while ((e = backupQ.pollFirst()) != null)
+                    ret.add(e);
+            }
 
-        if (!pending.isEmpty()) {
-            if (ret == null)
-                ret = new ArrayList<>();
+            if (!pending.isEmpty()) {
+                if (ret == null)
+                    ret = new ArrayList<>();
 
-            ret.addAll(pending.values());
+                ret.addAll(pending.values());
+            }
+
+            if (curBatch.compareAndSet(batch, null))
+                break;
         }
 
         return ret;
@@ -142,21 +155,30 @@ public class CacheContinuousQueryEventBuffer {
     private Object process0(long cntr, CacheContinuousQueryEntry entry, boolean backup) {
         assert cntr >= 0 : cntr;
 
-        Batch batch = initBatch(entry.topologyVersion());
+        Batch batch;
+        Object res = null;
 
-        if (batch == null || cntr < batch.startCntr) {
-            if (backup)
-                backupQ.add(entry);
+        for (;;) {
+            batch = initBatch(entry.topologyVersion());
 
-            return entry;
-        }
+            if (batch == null || cntr < batch.startCntr) {
+                if (backup)
+                    backupQ.add(entry);
 
-        Object res = null;
+                return entry;
+            }
+
+            if (cntr <= batch.endCntr) {
+                res = batch.processEntry0(null, cntr, entry, backup);
+
+                if (res == RETRY)
+                    continue;
+            }
+            else
+                pending.put(cntr, entry);
 
-        if (cntr <= batch.endCntr)
-            res = batch.processEvent0(null, cntr, entry, backup);
-        else
-            pending.put(cntr, entry);
+            break;
+        }
 
         Batch batch0 = curBatch.get();
 
@@ -166,7 +188,7 @@ public class CacheContinuousQueryEventBuffer {
 
                 res = processPending(res, batch, backup);
 
-                batch0 = curBatch.get();
+                batch0 = initBatch(entry.topologyVersion());
             }
             while (batch != batch0);
         }
@@ -184,17 +206,22 @@ public class CacheContinuousQueryEventBuffer {
         if (batch != null)
             return batch;
 
-        long curCntr = currentPartitionCounter();
+        for (;;) {
+            long curCntr = currentPartitionCounter();
 
-        if (curCntr == -1)
-            return null;
+            if (curCntr == -1)
+                return null;
 
-        batch = new Batch(curCntr + 1, 0L, new CacheContinuousQueryEntry[BUF_SIZE], topVer);
+            batch = new Batch(curCntr + 1, 0L, new CacheContinuousQueryEntry[BUF_SIZE], topVer);
 
-        if (curBatch.compareAndSet(null, batch))
-            return batch;
+            if (curBatch.compareAndSet(null, batch))
+                return batch;
+
+            batch = curBatch.get();
 
-        return curBatch.get();
+            if (batch != null)
+                return batch;
+        }
     }
 
     /**
@@ -211,7 +238,7 @@ public class CacheContinuousQueryEventBuffer {
                 assert cntr >= batch.startCntr && cntr <= batch.endCntr : cntr;
 
                 if (pending.remove(p.getKey()) != null)
-                    res = batch.processEvent0(res, p.getKey(), p.getValue(), backup);
+                    res = batch.processEntry0(res, p.getKey(), p.getValue(), backup);
             }
         }
 
@@ -235,7 +262,7 @@ public class CacheContinuousQueryEventBuffer {
         private int lastProc = -1;
 
         /** */
-        private final CacheContinuousQueryEntry[] entries;
+        private CacheContinuousQueryEntry[] entries;
 
         /** */
         private final AffinityTopologyVersion topVer;
@@ -261,7 +288,10 @@ public class CacheContinuousQueryEventBuffer {
         /**
          * @return Entries to send as part of backup queue.
          */
-        @Nullable synchronized List<CacheContinuousQueryEntry> backupFlushEntries() {
+        @Nullable synchronized List<CacheContinuousQueryEntry> flushAndReset() {
+            if (entries == null)
+                return null;
+
             List<CacheContinuousQueryEntry> res = null;
 
             long filtered = this.filtered;
@@ -283,15 +313,7 @@ public class CacheContinuousQueryEventBuffer {
                     if (e.isFiltered())
                         filtered++;
                     else {
-                        flushEntry = new CacheContinuousQueryEntry(e.cacheId(),
-                            e.eventType(),
-                            e.key(),
-                            e.value(),
-                            e.oldValue(),
-                            e.isKeepBinary(),
-                            e.partition(),
-                            e.updateCounter(),
-                            e.topologyVersion());
+                        flushEntry = e;
 
                         flushEntry.filteredCount(filtered);
 
@@ -316,6 +338,8 @@ public class CacheContinuousQueryEventBuffer {
                 res.add(filteredEntry(cntr - 1, filtered - 1));
             }
 
+            entries = null;
+
             return res;
         }
 
@@ -350,7 +374,7 @@ public class CacheContinuousQueryEventBuffer {
          * @return New result.
          */
         @SuppressWarnings("unchecked")
-        @Nullable private Object processEvent0(
+        @Nullable private Object processEntry0(
             @Nullable Object res,
             long cntr,
             CacheContinuousQueryEntry entry,
@@ -358,6 +382,9 @@ public class CacheContinuousQueryEventBuffer {
             int pos = (int)(cntr - startCntr);
 
             synchronized (this) {
+                if (entries == null)
+                    return RETRY;
+
                 entries[pos] = entry;
 
                 int next = lastProc + 1;
@@ -409,17 +436,24 @@ public class CacheContinuousQueryEventBuffer {
                     }
 
                     lastProc = pos;
-                }
-                else
-                    return res;
-            }
 
-            if (pos == entries.length -1) {
-                Arrays.fill(entries, null);
+                    if (pos == entries.length - 1) {
+                        Arrays.fill(entries, null);
+
+                        Batch nextBatch = new Batch(this.startCntr + BUF_SIZE,
+                            filtered,
+                            entries,
+                            entry.topologyVersion());
+
+                        entries = null;
 
-                Batch nextBatch = new Batch(this.startCntr + BUF_SIZE, filtered, entries, entry.topologyVersion());
+                        assert curBatch.get() == this;
 
-                curBatch.set(nextBatch);
+                        curBatch.set(nextBatch);
+                    }
+                }
+                else
+                    return res;
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/01f45c1b/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 b4f2dbd..ebfbe4f 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
@@ -452,7 +452,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
                     for (Map.Entry<Integer, CacheContinuousQueryEventBuffer> bufE : entryBufs.entrySet()) {
                         CacheContinuousQueryEventBuffer buf = bufE.getValue();
 
-                        Collection<CacheContinuousQueryEntry> backupQueue = buf.resetBackupQueue();
+                        Collection<CacheContinuousQueryEntry> backupQueue = buf.flushOnExchange();
 
                         if (backupQueue != null && node != null) {
                             for (CacheContinuousQueryEntry e : backupQueue) {
@@ -958,7 +958,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
         for (Map.Entry<Integer, CacheContinuousQueryEventBuffer> bufE : entryBufs.entrySet()) {
             CacheContinuousQueryEventBuffer buf = bufE.getValue();
 
-            buf.resetBackupQueue();
+            buf.flushOnExchange();
         }
     }
 


[22/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: ab5aead4dcb651001c362326e6a0b50350b31c2e
Parents: ff0a2dd
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 25 17:33:46 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 25 17:33:46 2017 +0300

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryEntry.java   | 59 ++++++++++----------
 .../CacheContinuousQueryEventBuffer.java        |  7 +++
 2 files changed, 35 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab5aead4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
index 9db92b2..28fdee3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
@@ -51,6 +51,9 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
     private static final byte FILTERED_ENTRY = 0b0010;
 
     /** */
+    private static final byte KEEP_BINARY = 0b0100;
+
+    /** */
     private static final EventType[] EVT_TYPE_VALS = EventType.values();
 
     /**
@@ -105,9 +108,6 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
     @GridToStringInclude
     private AffinityTopologyVersion topVer;
 
-    /** Keep binary. */
-    private boolean keepBinary;
-
     /** */
     private long filteredCnt;
 
@@ -124,6 +124,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
      * @param key Key.
      * @param newVal New value.
      * @param oldVal Old value.
+     * @param keepBinary Keep binary flag.
      * @param part Partition.
      * @param updateCntr Update partition counter.
      * @param topVer Topology version if applicable.
@@ -146,7 +147,9 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
         this.part = part;
         this.updateCntr = updateCntr;
         this.topVer = topVer;
-        this.keepBinary = keepBinary;
+
+        if (keepBinary)
+            flags |= KEEP_BINARY;
     }
 
     /**
@@ -231,7 +234,15 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
             return this;
 
         CacheContinuousQueryEntry e = new CacheContinuousQueryEntry(
-                cacheId, null, null, null, null, keepBinary, part, updateCntr, topVer);
+            cacheId,
+            null,
+            null,
+            null,
+            null,
+            false,
+            part,
+            updateCntr,
+            topVer);
 
         e.flags = flags;
 
@@ -256,7 +267,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
      * @return Keep binary flag.
      */
     boolean isKeepBinary() {
-        return keepBinary;
+        return (flags & KEEP_BINARY) != 0;
     }
 
     /**
@@ -370,42 +381,36 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeBoolean("keepBinary", keepBinary))
-                    return false;
-
-                writer.incrementState();
-
-            case 5:
                 if (!writer.writeMessage("key", isFiltered() ? null : key))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 5:
                 if (!writer.writeMessage("newVal", isFiltered() ? null : newVal))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 6:
                 if (!writer.writeMessage("oldVal", isFiltered() ? null : oldVal))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
+            case 7:
                 if (!writer.writeInt("part", part))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 8:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 9:
                 if (!writer.writeLong("updateCntr", updateCntr))
                     return false;
 
@@ -457,14 +462,6 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 4:
-                keepBinary = reader.readBoolean("keepBinary");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 5:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -472,7 +469,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
                 reader.incrementState();
 
-            case 6:
+            case 5:
                 newVal = reader.readMessage("newVal");
 
                 if (!reader.isLastRead())
@@ -480,7 +477,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
                 reader.incrementState();
 
-            case 7:
+            case 6:
                 oldVal = reader.readMessage("oldVal");
 
                 if (!reader.isLastRead())
@@ -488,7 +485,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
                 reader.incrementState();
 
-            case 8:
+            case 7:
                 part = reader.readInt("part");
 
                 if (!reader.isLastRead())
@@ -496,7 +493,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
                 reader.incrementState();
 
-            case 9:
+            case 8:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -504,7 +501,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
                 reader.incrementState();
 
-            case 10:
+            case 9:
                 updateCntr = reader.readLong("updateCntr");
 
                 if (!reader.isLastRead())
@@ -519,7 +516,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 11;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab5aead4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index f496c8c..fd4029c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -96,6 +96,13 @@ public class CacheContinuousQueryEventBuffer {
         else
             ret = entries;
 
+        if (!pending.isEmpty()) {
+            if (ret == null)
+                ret = new ArrayList<>();
+
+            ret.addAll(pending.values());
+        }
+
         return ret;
     }
 


[17/39] ignite git commit: IGNITE-5284: Splitted IgniteH2Indexing into several classes. This closes #1999.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
new file mode 100644
index 0000000..75d739d
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.GridStringBuilder;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.h2.engine.Session;
+import org.h2.jdbc.JdbcConnection;
+import org.h2.result.SortOrder;
+import org.h2.table.IndexColumn;
+
+import java.lang.reflect.Constructor;
+import java.sql.Connection;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * H2 utility methods.
+ */
+public class H2Utils {
+    /** Spatial index class name. */
+    private static final String SPATIAL_IDX_CLS =
+        "org.apache.ignite.internal.processors.query.h2.opt.GridH2SpatialIndex";
+
+    /** */
+    public static final char ESC_CH = '\"';
+
+    /** */
+    private static final String ESC_STR = ESC_CH + "" + ESC_CH;
+
+    /**
+     * @param c1 First column.
+     * @param c2 Second column.
+     * @return {@code true} If they are the same.
+     */
+    public static boolean equals(IndexColumn c1, IndexColumn c2) {
+        return c1.column.getColumnId() == c2.column.getColumnId();
+    }
+
+    /**
+     * @param cols Columns list.
+     * @param col Column to find.
+     * @return {@code true} If found.
+     */
+    public static boolean containsColumn(List<IndexColumn> cols, IndexColumn col) {
+        for (int i = cols.size() - 1; i >= 0; i--) {
+            if (equals(cols.get(i), col))
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Check whether columns list contains key or key alias column.
+     *
+     * @param desc Row descriptor.
+     * @param cols Columns list.
+     * @return Result.
+     */
+    public static boolean containsKeyColumn(GridH2RowDescriptor desc, List<IndexColumn> cols) {
+        for (int i = cols.size() - 1; i >= 0; i--) {
+            if (desc.isKeyColumn(cols.get(i).column.getColumnId()))
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Generate {@code CREATE INDEX} SQL statement for given params.
+     * @param fullTblName Fully qualified table name.
+     * @param h2Idx H2 index.
+     * @param ifNotExists Quietly skip index creation if it exists.
+     * @return Statement string.
+     */
+    public static String indexCreateSql(String fullTblName, GridH2IndexBase h2Idx, boolean ifNotExists,
+        boolean escapeAll) {
+        boolean spatial = F.eq(SPATIAL_IDX_CLS, h2Idx.getClass().getName());
+
+        GridStringBuilder sb = new SB("CREATE ")
+            .a(spatial ? "SPATIAL " : "")
+            .a("INDEX ")
+            .a(ifNotExists ? "IF NOT EXISTS " : "")
+            .a(escapeName(h2Idx.getName(), escapeAll))
+            .a(" ON ")
+            .a(fullTblName)
+            .a(" (");
+
+        boolean first = true;
+
+        for (IndexColumn col : h2Idx.getIndexColumns()) {
+            if (first)
+                first = false;
+            else
+                sb.a(", ");
+
+            sb.a("\"" + col.columnName + "\"").a(" ").a(col.sortType == SortOrder.ASCENDING ? "ASC" : "DESC");
+        }
+
+        sb.a(')');
+
+        return sb.toString();
+    }
+
+    /**
+     * Generate {@code CREATE INDEX} SQL statement for given params.
+     * @param schemaName <b>Quoted</b> schema name.
+     * @param idxName Index name.
+     * @param ifExists Quietly skip index drop if it exists.
+     * @param escapeAll Escape flag.
+     * @return Statement string.
+     */
+    public static String indexDropSql(String schemaName, String idxName, boolean ifExists, boolean escapeAll) {
+        return "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + schemaName + '.' + escapeName(idxName, escapeAll);
+    }
+
+    /**
+     * Escapes name to be valid SQL identifier. Currently just replaces '.' and '$' sign with '_'.
+     *
+     * @param name Name.
+     * @param escapeAll Escape flag.
+     * @return Escaped name.
+     */
+    public static String escapeName(String name, boolean escapeAll) {
+        if (name == null) // It is possible only for a cache name.
+            return ESC_STR;
+
+        if (escapeAll)
+            return ESC_CH + name + ESC_CH;
+
+        SB sb = null;
+
+        for (int i = 0; i < name.length(); i++) {
+            char ch = name.charAt(i);
+
+            if (!Character.isLetter(ch) && !Character.isDigit(ch) && ch != '_' &&
+                !(ch == '"' && (i == 0 || i == name.length() - 1)) && ch != '-') {
+                // Class name can also contain '$' or '.' - these should be escaped.
+                assert ch == '$' || ch == '.';
+
+                if (sb == null)
+                    sb = new SB();
+
+                sb.a(name.substring(sb.length(), i));
+
+                // Replace illegal chars with '_'.
+                sb.a('_');
+            }
+        }
+
+        if (sb == null)
+            return name;
+
+        sb.a(name.substring(sb.length(), name.length()));
+
+        return sb.toString();
+    }
+
+    /**
+     * @param desc Row descriptor.
+     * @param cols Columns list.
+     * @param keyCol Primary key column.
+     * @param affCol Affinity key column.
+     * @return The same list back.
+     */
+    public static List<IndexColumn> treeIndexColumns(GridH2RowDescriptor desc, List<IndexColumn> cols,
+        IndexColumn keyCol, IndexColumn affCol) {
+        assert keyCol != null;
+
+        if (!containsKeyColumn(desc, cols))
+            cols.add(keyCol);
+
+        if (affCol != null && !containsColumn(cols, affCol))
+            cols.add(affCol);
+
+        return cols;
+    }
+
+    /**
+     * Create spatial index.
+     *
+     * @param tbl Table.
+     * @param idxName Index name.
+     * @param cols Columns.
+     */
+    public static GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, IndexColumn[] cols) {
+        try {
+            Class<?> cls = Class.forName(SPATIAL_IDX_CLS);
+
+            Constructor<?> ctor = cls.getConstructor(
+                GridH2Table.class,
+                String.class,
+                Integer.TYPE,
+                IndexColumn[].class);
+
+            if (!ctor.isAccessible())
+                ctor.setAccessible(true);
+
+            final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
+
+            return (GridH2IndexBase)ctor.newInstance(tbl, idxName, segments, cols);
+        }
+        catch (Exception e) {
+            throw new IgniteException("Failed to instantiate: " + SPATIAL_IDX_CLS, e);
+        }
+    }
+
+    /**
+     * Stores rule for constructing schemaName according to cache configuration.
+     *
+     * @param ccfg Cache configuration.
+     * @return Proper schema name according to ANSI-99 standard.
+     */
+    public static String schemaNameFromCacheConfiguration(CacheConfiguration<?, ?> ccfg) {
+        if (ccfg.getSqlSchema() == null)
+            return escapeName(ccfg.getName(), true);
+
+        if (ccfg.getSqlSchema().charAt(0) == ESC_CH)
+            return ccfg.getSqlSchema();
+
+        return ccfg.isSqlEscapeAll() ?
+            escapeName(ccfg.getSqlSchema(), true) : ccfg.getSqlSchema().toUpperCase();
+    }
+
+    /**
+     * @param rsMeta Metadata.
+     * @return List of fields metadata.
+     * @throws SQLException If failed.
+     */
+    public static List<GridQueryFieldMetadata> meta(ResultSetMetaData rsMeta) throws SQLException {
+        List<GridQueryFieldMetadata> meta = new ArrayList<>(rsMeta.getColumnCount());
+
+        for (int i = 1; i <= rsMeta.getColumnCount(); i++) {
+            String schemaName = rsMeta.getSchemaName(i);
+            String typeName = rsMeta.getTableName(i);
+            String name = rsMeta.getColumnLabel(i);
+            String type = rsMeta.getColumnClassName(i);
+
+            if (type == null) // Expression always returns NULL.
+                type = Void.class.getName();
+
+            meta.add(new H2SqlFieldMetadata(schemaName, typeName, name, type));
+        }
+
+        return meta;
+    }
+
+    /**
+     * @param c Connection.
+     * @return Session.
+     */
+    public static Session session(Connection c) {
+        return (Session)((JdbcConnection)c).getSession();
+    }
+
+    /**
+     * @param conn Connection to use.
+     * @param distributedJoins If distributed joins are enabled.
+     * @param enforceJoinOrder Enforce join order of tables.
+     */
+    public static void setupConnection(Connection conn, boolean distributedJoins, boolean enforceJoinOrder) {
+        Session s = session(conn);
+
+        s.setForceJoinOrder(enforceJoinOrder);
+        s.setJoinBatchEnabled(distributedJoins);
+    }
+
+    /**
+     * Private constructor.
+     */
+    private H2Utils() {
+        // No-op.
+    }
+}


[15/39] ignite git commit: IGNITE-5284: Splitted IgniteH2Indexing into several classes. This closes #1999.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 9f01346..8fcd7dc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -37,6 +37,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.cache.query.QueryTable;
+import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -47,7 +48,6 @@ import org.h2.command.dml.SelectUnion;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.setupConnection;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2CollocationModel.isCollocated;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst.TRUE;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG;
@@ -1413,7 +1413,7 @@ public class GridSqlQuerySplitter {
         boolean distributedJoins,
         boolean enforceJoinOrder
     ) throws SQLException, IgniteCheckedException {
-        setupConnection(c, distributedJoins, enforceJoinOrder);
+        H2Utils.setupConnection(c, distributedJoins, enforceJoinOrder);
 
         try (PreparedStatement s = c.prepareStatement(qry)) {
             h2.bindParameters(s, F.asList(params));

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 1d8a762..6fff8de 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshalla
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.QueryTable;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
@@ -87,7 +88,6 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.QUERY_POOL;
 import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
-import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.setupConnection;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.distributedJoinMode;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.MAP;
@@ -581,7 +581,7 @@ public class GridMapQueryExecutor {
 
             Connection conn = h2.connectionForCache(mainCctx.name());
 
-            setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder);
+            H2Utils.setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder);
 
             GridH2QueryContext.set(qctx);
 
@@ -608,7 +608,7 @@ public class GridMapQueryExecutor {
                     // If we are not the target node for this replicated query, just ignore it.
                     if (qry.node() == null ||
                         (segmentId == 0 && qry.node().equals(ctx.localNodeId()))) {
-                        rs = h2.executeSqlQueryWithTimer(mainCctx.name(), conn, qry.query(),
+                        rs = h2.executeSqlQueryWithTimer(h2.schema(mainCctx.name()), conn, qry.query(),
                             F.asList(qry.parameters(params)), true,
                             timeout,
                             qr.cancels[qryIdx]);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8c75e4de/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 1ecbb40..a31263f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -65,6 +65,8 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
+import org.apache.ignite.internal.processors.query.h2.H2FieldsIterator;
+import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSortColumn;
@@ -100,7 +102,6 @@ import static java.util.Collections.singletonList;
 import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery.EMPTY_PARAMS;
-import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.setupConnection;
 import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter.mergeTableIdentifier;
@@ -787,18 +788,20 @@ public class GridReduceQueryExecutor {
 
                         UUID locNodeId = ctx.localNodeId();
 
-                        setupConnection(r.conn, false, enforceJoinOrder);
+                        H2Utils.setupConnection(r.conn, false, enforceJoinOrder);
 
                         GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, qryReqId, REDUCE)
                             .pageSize(r.pageSize).distributedJoinMode(OFF));
 
                         try {
+                            String schema = h2.schema(cacheName);
+
                             if (qry.explain())
-                                return explainPlan(r.conn, cacheName, qry, params);
+                                return explainPlan(r.conn, schema, qry, params);
 
                             GridCacheSqlQuery rdc = qry.reduceQuery();
 
-                            ResultSet res = h2.executeSqlQueryWithTimer(cacheName,
+                            ResultSet res = h2.executeSqlQueryWithTimer(schema,
                                 r.conn,
                                 rdc.query(),
                                 F.asList(rdc.parameters(params)),
@@ -806,7 +809,7 @@ public class GridReduceQueryExecutor {
                                 timeoutMillis,
                                 cancel);
 
-                            resIter = new IgniteH2Indexing.FieldsIterator(res);
+                            resIter = new H2FieldsIterator(res);
                         }
                         finally {
                             GridH2QueryContext.clearThreadLocal();
@@ -1213,18 +1216,18 @@ public class GridReduceQueryExecutor {
 
     /**
      * @param c Connection.
-     * @param cacheName Cache name.
+     * @param schema Schema.
      * @param qry Query.
      * @param params Query parameters.
      * @return Cursor for plans.
      * @throws IgniteCheckedException if failed.
      */
-    private Iterator<List<?>> explainPlan(JdbcConnection c, String cacheName, GridCacheTwoStepQuery qry, Object[] params)
+    private Iterator<List<?>> explainPlan(JdbcConnection c, String schema, GridCacheTwoStepQuery qry, Object[] params)
         throws IgniteCheckedException {
         List<List<?>> lists = new ArrayList<>();
 
         for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) {
-            ResultSet rs = h2.executeSqlQueryWithTimer(cacheName, c,
+            ResultSet rs = h2.executeSqlQueryWithTimer(schema, c,
                 "SELECT PLAN FROM " + mergeTableIdentifier(i), null, false, 0, null);
 
             lists.add(F.asList(getPlan(rs)));
@@ -1240,7 +1243,7 @@ public class GridReduceQueryExecutor {
 
         GridCacheSqlQuery rdc = qry.reduceQuery();
 
-        ResultSet rs = h2.executeSqlQueryWithTimer(cacheName,
+        ResultSet rs = h2.executeSqlQueryWithTimer(schema,
             c,
             "EXPLAIN " + rdc.query(),
             F.asList(rdc.parameters(params)),


[26/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 8cbcef657d2958395377654d44779c68ce16fca6
Parents: 79e34c2
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 25 20:33:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 25 20:33:50 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheMapEntry.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8cbcef65/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 30c2a33..80f872c 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
@@ -1838,6 +1838,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (updateRes.success())
                 updateMetrics(c.op, metrics);
 
+            lsnrs = cctx.continuousQueries().updateListeners(internal, false);
+
             // Continuous query filter should be perform under lock.
             if (lsnrs != null) {
                 CacheObject evtVal = cctx.unwrapTemporary(updateVal);


[37/39] ignite git commit: Merge branch 'ignite-5075-cc' into ignite-5075-cc-debug

Posted by sb...@apache.org.
Merge branch 'ignite-5075-cc' into ignite-5075-cc-debug

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 7e56f3ae8cb616018cf267e9f4f16e66482f7ea5
Parents: 58919c5 be43bf8
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 16:45:51 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 16:45:51 2017 +0300

----------------------------------------------------------------------
 doap_Ignite.rdf                                 |    2 +
 .../examples/datagrid/CacheQueryExample.java    |    2 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    3 +
 .../jdbc/thin/JdbcConnectionSelfTest.java       |  195 ++
 .../src/main/java/org/apache/ignite/Ignite.java |   18 +-
 .../org/apache/ignite/IgniteJdbcThinDriver.java |  312 +++
 .../configuration/CacheConfiguration.java       |    5 +-
 .../MemoryPolicyConfiguration.java              |   96 +-
 .../ignite/internal/GridKernalContext.java      |    8 +-
 .../ignite/internal/GridKernalContextImpl.java  |   12 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   27 +
 .../apache/ignite/internal/IgniteKernal.java    |   42 +-
 .../internal/binary/BinaryWriterExImpl.java     |   16 +-
 .../internal/jdbc/thin/JdbcConnection.java      |  529 +++++
 .../ignite/internal/jdbc/thin/JdbcTcpIo.java    |  207 ++
 .../ignite/internal/jdbc2/JdbcConnection.java   |    3 +-
 .../managers/communication/GridIoManager.java   |  380 +++-
 .../communication/IgniteIoTestMessage.java      |  362 ++-
 .../managers/indexing/GridIndexingManager.java  |   19 +-
 .../processors/cache/ClusterCachesInfo.java     |   10 +-
 .../processors/cache/ExchangeActions.java       |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    6 +-
 .../processors/cache/GridCacheMapEntry.java     |    7 +-
 .../GridCachePartitionExchangeManager.java      |    2 +-
 .../processors/cache/GridCacheProcessor.java    |   47 +-
 .../processors/cache/IgniteCacheProxy.java      |   15 +-
 .../IgniteCacheDatabaseSharedManager.java       |   40 +
 .../cache/database/MemoryMetricsImpl.java       |   31 +-
 .../cache/database/tree/BPlusTree.java          |   64 +-
 .../dht/GridClientPartitionTopology.java        |   31 +-
 .../dht/GridDhtPartitionTopology.java           |    9 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   59 +-
 .../GridDhtPartitionsExchangeFuture.java        |   51 +-
 .../cache/query/GridCacheQueryManager.java      |   32 +-
 .../continuous/CacheContinuousQueryEntry.java   |   73 +-
 .../CacheContinuousQueryEventBuffer.java        |  224 +-
 .../continuous/CacheContinuousQueryHandler.java |    4 +-
 .../continuous/CacheContinuousQueryManager.java |   16 +-
 .../CacheContinuousQueryPartitionRecovery.java  |    2 +-
 .../processors/odbc/OdbcNioListener.java        |  242 --
 .../internal/processors/odbc/OdbcProcessor.java |  199 --
 .../odbc/SqlListenerAbstractMessageParser.java  |  265 +++
 .../odbc/SqlListenerAbstractObjectReader.java   |  137 ++
 .../odbc/SqlListenerAbstractObjectWriter.java   |  111 +
 .../processors/odbc/SqlListenerNioListener.java |  263 +++
 .../processors/odbc/SqlListenerProcessor.java   |  191 ++
 .../odbc/SqlListenerRequestHandlerImpl.java     |  494 ++++
 .../processors/odbc/jdbc/JdbcMessageParser.java |   50 +
 .../processors/odbc/jdbc/JdbcObjectReader.java  |   33 +
 .../processors/odbc/jdbc/JdbcObjectWriter.java  |   33 +
 .../processors/odbc/odbc/OdbcMessageParser.java |  249 +--
 .../processors/odbc/odbc/OdbcObjectReader.java  |   33 +
 .../processors/odbc/odbc/OdbcObjectWriter.java  |   32 +
 .../odbc/odbc/OdbcRequestHandler.java           |  513 -----
 .../processors/query/GridQueryIndexing.java     |   96 +-
 .../processors/query/GridQueryProcessor.java    |  232 +-
 .../processors/query/QueryIndexKey.java         |   20 +-
 .../query/QueryTypeDescriptorImpl.java          |   16 +-
 .../processors/query/QueryTypeIdKey.java        |   26 +-
 .../processors/query/QueryTypeNameKey.java      |   12 +-
 .../internal/processors/query/QueryUtils.java   |   14 +-
 .../schema/SchemaIndexCacheVisitorImpl.java     |   12 +-
 .../operation/SchemaAbstractOperation.java      |    2 +-
 .../handlers/GridRestCommandHandlerAdapter.java |    3 +
 .../handlers/cache/GridCacheCommandHandler.java |   10 +-
 .../handlers/query/QueryCommandHandler.java     |   13 +-
 .../spi/indexing/IndexingQueryFilter.java       |    8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   17 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |    6 +-
 ...iteMarshallerCacheClassNameConflictTest.java |    9 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |    6 +-
 .../cache/MemoryPolicyConfigValidationTest.java |   62 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |   14 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |    6 +-
 .../CacheContinuousQueryEventBufferTest.java    |    7 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |    4 +-
 .../processors/igfs/IgfsIgniteMock.java         |   24 +
 .../odbc/OdbcProcessorValidationSelfTest.java   |  182 --
 .../SqlListenerProcessorValidationSelfTest.java |  184 ++
 .../ignite/testframework/GridTestUtils.java     |    8 +-
 .../config/GridTestProperties.java              |    3 +-
 .../ignite/testframework/junits/IgniteMock.java |    5 +
 .../junits/multijvm/IgniteProcessProxy.java     |   18 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |   17 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   75 +-
 ...iteBinarySimpleNameMapperBasicTestSuite.java |    3 +-
 .../query/h2/DmlStatementsProcessor.java        |   71 +-
 .../query/h2/GridH2ResultSetIterator.java       |  191 --
 .../query/h2/H2ConnectionWrapper.java           |   67 +
 .../processors/query/h2/H2DatabaseType.java     |  161 ++
 .../processors/query/h2/H2FieldsIterator.java   |   50 +
 .../processors/query/h2/H2KeyValueIterator.java |   48 +
 .../query/h2/H2ResultSetIterator.java           |  191 ++
 .../processors/query/h2/H2RowDescriptor.java    |  479 ++++
 .../internal/processors/query/h2/H2Schema.java  |  135 ++
 .../processors/query/h2/H2SqlFieldMetadata.java |  111 +
 .../processors/query/h2/H2StatementCache.java   |   73 +
 .../processors/query/h2/H2TableDescriptor.java  |  345 +++
 .../processors/query/h2/H2TableEngine.java      |   89 +
 .../query/h2/H2TwoStepCachedQuery.java          |   49 +
 .../query/h2/H2TwoStepCachedQueryKey.java       |  107 +
 .../internal/processors/query/h2/H2Utils.java   |  299 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 2112 +++---------------
 .../query/h2/database/H2PkHashIndex.java        |    4 +-
 .../query/h2/database/H2TreeIndex.java          |    4 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |   55 +-
 .../query/h2/opt/GridH2IndexBase.java           |   10 +-
 .../processors/query/h2/opt/GridH2Table.java    |    2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |    2 +-
 .../query/h2/opt/GridLuceneIndex.java           |   15 +-
 .../query/h2/sql/GridSqlCreateTable.java        |  153 ++
 .../query/h2/sql/GridSqlDropTable.java          |   79 +
 .../query/h2/sql/GridSqlQueryParser.java        |  260 +++
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    8 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   39 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   12 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |   18 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |    8 +-
 .../cache/index/H2DynamicTableSelfTest.java     |  320 +++
 .../query/IgniteQueryDedicatedPoolTest.java     |    6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  140 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  286 ++-
 .../IgniteCacheQuerySelfTestSuite.java          |    2 +
 .../cpp/odbc/include/ignite/odbc/message.h      |   10 +
 .../Impl/CacheFieldsQueryExecutor.cs            |   39 -
 .../Impl/CacheQueryableBase.cs                  |   14 +-
 .../Impl/ICacheQueryableInternal.cs             |    6 -
 .../org/apache/ignite/IgniteSpringBean.java     |    7 +
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../org/apache/ignite/yardstick/IgniteNode.java |    5 +
 .../apache/ignite/yardstick/io/FileUtils.java   |   64 +
 132 files changed, 8742 insertions(+), 4341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7e56f3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e56f3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index 5346144,7d33614..d0950ef
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@@ -28,9 -27,8 +27,10 @@@ import java.util.concurrent.ConcurrentS
  import java.util.concurrent.atomic.AtomicReference;
  import org.apache.ignite.IgniteSystemProperties;
  import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 +import org.apache.ignite.spi.communication.tcp.TestDebugLog;
 +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
  import org.jetbrains.annotations.Nullable;
+ import org.jsr166.ConcurrentLinkedDeque8;
  
  /**
   *
@@@ -85,27 -83,31 +85,39 @@@ public class CacheContinuousQueryEventB
          Batch batch = curBatch.get();
  
          if (batch != null)
-             entries = batch.backupFlushEntries();
+             ret = batch.flushCurrentEntries();
+ 
+         int size = backupQ.sizex();
  
-         if (!backupQ.isEmpty()) {
-             if (entries != null)
-                 backupQ.addAll(entries);
+         if (size > 0) {
+             if (ret == null)
+                 ret = new ArrayList<>();
+ 
+             for (int i = 0; i < size; i++) {
+                 CacheContinuousQueryEntry e = backupQ.pollFirst();
+ 
+                 if (e != null)
+                     ret.add(e);
+                 else
+                     break;
+             }
+         }
  
-             ret = this.backupQ;
+         if (!pending.isEmpty()) {
+             if (ret == null)
+                 ret = new ArrayList<>();
  
-             backupQ = new ConcurrentLinkedDeque<>();
+             ret.addAll(pending.values());
          }
-         else
-             ret = entries;
  
 +        if (ret != null) {
 +            for (CacheContinuousQueryEntry e : ret)
 +                TestDebugLog.addEntryMessage(part,
 +                    e.updateCounter(),
 +                    "filtered " + e.filteredCount() +
 +                        " reset backup");
 +        }
 +
          return ret;
      }
  
@@@ -145,33 -147,29 +157,39 @@@
      private Object process0(long cntr, CacheContinuousQueryEntry entry, boolean backup) {
          assert cntr >= 0 : cntr;
  
-         Batch batch = initBatch(entry.topologyVersion());
+         Batch batch;
+         Object res = null;
  
-         if (batch == null || cntr < batch.startCntr) {
-             if (backup)
-                 backupQ.add(entry);
+         for (;;) {
+             batch = initBatch(entry.topologyVersion());
+ 
+             if (batch == null || cntr < batch.startCntr) {
+                 if (backup)
+                     backupQ.add(entry);
  
 -                return entry;
 -            }
 +            TestDebugLog.addEntryMessage(part,
 +                cntr,
 +                "buffer rcd small start=" + batch.startCntr +
 +                    " cntr=" + cntr +
 +                    ", backup=" + backup +
-                     " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());
- 
-             return entry;
++                    " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());return entry;
 +        }
  
-         Object res = null;
+             if (cntr <= batch.endCntr) {
+                 res = batch.processEntry0(null, cntr, entry, backup);
  
-         if (cntr <= batch.endCntr)
-             res = batch.processEvent0(null, cntr, entry, backup);
-         else {
 -                if (res == RETRY)
++        if (
++            res = = RETRY)
+                     continue;
+             }
 -            else
 -                pending.put(cntr, entry);
++        else{
 +            TestDebugLog.addEntryMessage(part,
 +                cntr,
 +                "buffer add pending start=" + batch.startCntr +
 +                    " cntr=" + cntr +
-                     " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());
-             pending.put(cntr, entry);
++                    " topVer=" + ((CacheContinuousQueryEntry)entry).topologyVersion());pending.put(cntr, entry);}
+ 
+             break;
          }
  
          Batch batch0 = curBatch.get();
@@@ -200,19 -198,22 +218,22 @@@
          if (batch != null)
              return batch;
  
-         long curCntr = currentPartitionCounter();
+         for (;;) {
+             long curCntr = currentPartitionCounter();
  
-         if (curCntr == -1)
-             return null;
+             if (curCntr == -1)
+                 return null;
  
-         TestDebugLog.addEntryMessage(part, curCntr, "created batch");
 -            batch = new Batch(curCntr + 1, 0L, new CacheContinuousQueryEntry[BUF_SIZE], topVer);
++        TestDebugLog.addEntryMessage(part, curCntr, "created batch");batch = new Batch(curCntr + 1, 0L, new CacheContinuousQueryEntry[BUF_SIZE], topVer);
  
-         batch = new Batch(curCntr + 1, 0L, new CacheContinuousQueryEntry[BUF_SIZE], topVer);
+             if (curBatch.compareAndSet(null, batch))
+                 return batch;
  
-         if (curBatch.compareAndSet(null, batch))
-             return batch;
+             batch = curBatch.get();
  
-         return curBatch.get();
+             if (batch != null)
+                 return batch;
+         }
      }
  
      /**
@@@ -376,13 -423,9 +443,16 @@@
              int pos = (int)(cntr - startCntr);
  
              synchronized (this) {
+                 if (entries == null)
+                     return RETRY;
+ 
 +                TestDebugLog.addEntryMessage(part,
 +                    cntr,
 +                    "buffer process start=" + startCntr +
 +                        ", lastProc=" + lastProc +
 +                        " pos=" + pos +
 +                        " topVer=" + entry.topologyVersion());
 +
                  entries[pos] = entry;
  
                  int next = lastProc + 1;
@@@ -405,45 -440,11 +475,20 @@@
  
                                  filtered = 0;
  
-                                 if (res == null) {
-                                     if (backup)
-                                         backupQ.add(entry0);
-                                     else
-                                         res = entry0;
-                                 }
-                                 else {
-                                     assert !backup;
- 
-                                     List<CacheContinuousQueryEntry> resList;
- 
-                                     if (res instanceof CacheContinuousQueryEntry) {
-                                         resList = new ArrayList<>();
- 
-                                         resList.add((CacheContinuousQueryEntry)res);
-                                     }
-                                     else {
-                                         assert res instanceof List : res;
- 
-                                         resList = (List<CacheContinuousQueryEntry>)res;
-                                     }
- 
-                                     resList.add(entry0);
- 
-                                     res = resList;
-                                 }
+                                 res = addResult(res, entry0, backup);
                              }
 -                            else
 +                            else {
                                  filtered++;
  
 +                                TestDebugLog.addEntryMessage(part,
 +                                    cntr,
 +                                    "buffer process inc filtered start=" + startCntr +
 +                                        ", lastProc=" + lastProc +
 +                                        " pos=" + pos +
 +                                        ", filtered=" + filtered +
 +                                        " topVer=" + entry0.topologyVersion());
 +                            }
 +
                              pos = i;
                          }
                          else

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e56f3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e56f3ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryPartitionRecovery.java
----------------------------------------------------------------------


[28/39] ignite git commit: Updated Ignite's DOAP file

Posted by sb...@apache.org.
Updated Ignite's DOAP file


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: bdd43ff50674acaec5048a9253f911106dfe24b6
Parents: 7325232
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu May 25 11:29:27 2017 -0700
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu May 25 11:29:27 2017 -0700

----------------------------------------------------------------------
 doap_Ignite.rdf | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd43ff5/doap_Ignite.rdf
----------------------------------------------------------------------
diff --git a/doap_Ignite.rdf b/doap_Ignite.rdf
index 3dc641b..16649c4 100644
--- a/doap_Ignite.rdf
+++ b/doap_Ignite.rdf
@@ -44,6 +44,8 @@
     <category rdf:resource="http://projects.apache.org/category/sql" />
     <category rdf:resource="http://projects.apache.org/category/cloud" />
     <category rdf:resource="http://projects.apache.org/category/database" />
+    <category rdf:resource="http://projects.apache.org/category/distributed-sql-database" />
+    <category rdf:resource="http://projects.apache.org/category/data-management-platform" />
     <category rdf:resource="http://projects.apache.org/category/hadoop" />
     <category rdf:resource="http://projects.apache.org/category/network-server" />
     <category rdf:resource="http://projects.apache.org/category/network-client" />


[12/39] ignite git commit: Minor - corrected TODO

Posted by sb...@apache.org.
Minor - corrected TODO


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: b131ff0a5470096522f960619513cfaa0d6f136b
Parents: a3908bd
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed May 24 10:19:49 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed May 24 10:19:49 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheMapEntry.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b131ff0a/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 6b7d6f5..15e4469 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
@@ -3448,7 +3448,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             // Version has changed since entry passed the filter. Do it again.
                             continue;
 
-                        // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost.
+                        // TODO IGNITE-5286: need keep removed entries in heap map, otherwise removes can be lost.
                         if (cctx.deferredDelete() && deletedUnlocked())
                             return false;
 


[34/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 6c73a36671d10db7739c76fc4018d4b43f83cb5b
Parents: e24b538
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 14:11:23 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 14:11:23 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheIoManager.java   | 6 +++++-
 .../query/continuous/CacheContinuousQueryEventBuffer.java      | 3 +--
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6c73a366/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 fdd29e4..bb31645 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
@@ -874,7 +874,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         finally {
             // Reset thread local context.
             cctx.tm().resetContext();
-            cctx.mvcc().contextReset();
+
+            GridCacheMvccManager mvcc = cctx.mvcc();
+
+            if (mvcc != null)
+                mvcc.contextReset();
 
             // Unwind eviction notifications.
             if (msg instanceof IgniteTxStateAware) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c73a366/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index acc6d50..31b7ace 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -35,9 +35,8 @@ import org.jetbrains.annotations.Nullable;
  */
 public class CacheContinuousQueryEventBuffer {
     /** */
-    // TODO increase to 1000
     private static final int BUF_SIZE =
-        IgniteSystemProperties.getInteger("IGNITE_CONTINUOUS_QUERY_SERVER_BUFFER_SIZE", 5);
+        IgniteSystemProperties.getInteger("IGNITE_CONTINUOUS_QUERY_SERVER_BUFFER_SIZE", 1000);
 
     /** */
     private static final Object RETRY = new Object();


[31/39] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-5075-cc

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


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: f651e87ca7b1fbafc456eabc0a1b0b92734fc225
Parents: e3500de bdd43ff
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 11:32:48 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 11:32:48 2017 +0300

----------------------------------------------------------------------
 doap_Ignite.rdf                                 |    2 +
 .../examples/datagrid/CacheQueryExample.java    |    2 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    3 +
 .../jdbc/thin/JdbcConnectionSelfTest.java       |  195 ++
 .../src/main/java/org/apache/ignite/Ignite.java |   18 +-
 .../org/apache/ignite/IgniteJdbcThinDriver.java |  312 +++
 .../configuration/CacheConfiguration.java       |    5 +-
 .../MemoryPolicyConfiguration.java              |   96 +-
 .../ignite/internal/GridKernalContext.java      |    8 +-
 .../ignite/internal/GridKernalContextImpl.java  |   12 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   27 +
 .../apache/ignite/internal/IgniteKernal.java    |   42 +-
 .../internal/binary/BinaryWriterExImpl.java     |   16 +-
 .../internal/jdbc/thin/JdbcConnection.java      |  529 +++++
 .../ignite/internal/jdbc/thin/JdbcTcpIo.java    |  207 ++
 .../ignite/internal/jdbc2/JdbcConnection.java   |    3 +-
 .../managers/communication/GridIoManager.java   |  380 +++-
 .../communication/IgniteIoTestMessage.java      |  362 ++-
 .../managers/indexing/GridIndexingManager.java  |   19 +-
 .../processors/cache/ClusterCachesInfo.java     |   10 +-
 .../processors/cache/ExchangeActions.java       |    2 +-
 .../processors/cache/GridCacheMapEntry.java     |    4 +-
 .../processors/cache/GridCacheProcessor.java    |   47 +-
 .../processors/cache/IgniteCacheProxy.java      |   15 +-
 .../IgniteCacheDatabaseSharedManager.java       |   40 +
 .../cache/database/MemoryMetricsImpl.java       |   31 +-
 .../cache/database/tree/BPlusTree.java          |   64 +-
 .../cache/query/GridCacheQueryManager.java      |   32 +-
 .../processors/odbc/OdbcNioListener.java        |  242 --
 .../internal/processors/odbc/OdbcProcessor.java |  199 --
 .../odbc/SqlListenerAbstractMessageParser.java  |  265 +++
 .../odbc/SqlListenerAbstractObjectReader.java   |  137 ++
 .../odbc/SqlListenerAbstractObjectWriter.java   |  111 +
 .../processors/odbc/SqlListenerNioListener.java |  263 +++
 .../processors/odbc/SqlListenerProcessor.java   |  191 ++
 .../odbc/SqlListenerRequestHandlerImpl.java     |  494 ++++
 .../processors/odbc/jdbc/JdbcMessageParser.java |   50 +
 .../processors/odbc/jdbc/JdbcObjectReader.java  |   33 +
 .../processors/odbc/jdbc/JdbcObjectWriter.java  |   33 +
 .../processors/odbc/odbc/OdbcMessageParser.java |  249 +--
 .../processors/odbc/odbc/OdbcObjectReader.java  |   33 +
 .../processors/odbc/odbc/OdbcObjectWriter.java  |   32 +
 .../odbc/odbc/OdbcRequestHandler.java           |  513 -----
 .../processors/query/GridQueryIndexing.java     |   96 +-
 .../processors/query/GridQueryProcessor.java    |  232 +-
 .../processors/query/QueryIndexKey.java         |   20 +-
 .../query/QueryTypeDescriptorImpl.java          |   16 +-
 .../processors/query/QueryTypeIdKey.java        |   26 +-
 .../processors/query/QueryTypeNameKey.java      |   12 +-
 .../internal/processors/query/QueryUtils.java   |   14 +-
 .../schema/SchemaIndexCacheVisitorImpl.java     |   12 +-
 .../operation/SchemaAbstractOperation.java      |    2 +-
 .../handlers/GridRestCommandHandlerAdapter.java |    3 +
 .../handlers/cache/GridCacheCommandHandler.java |   10 +-
 .../handlers/query/QueryCommandHandler.java     |   13 +-
 .../spi/indexing/IndexingQueryFilter.java       |    8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   17 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |    6 +-
 ...iteMarshallerCacheClassNameConflictTest.java |    9 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |    6 +-
 .../cache/MemoryPolicyConfigValidationTest.java |   62 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |   14 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |    6 +-
 .../processors/igfs/IgfsIgniteMock.java         |   24 +
 .../odbc/OdbcProcessorValidationSelfTest.java   |  182 --
 .../SqlListenerProcessorValidationSelfTest.java |  184 ++
 .../ignite/testframework/GridTestUtils.java     |    8 +-
 .../config/GridTestProperties.java              |    3 +-
 .../ignite/testframework/junits/IgniteMock.java |    5 +
 .../junits/multijvm/IgniteProcessProxy.java     |   18 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |   17 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   75 +-
 ...iteBinarySimpleNameMapperBasicTestSuite.java |    3 +-
 .../query/h2/DmlStatementsProcessor.java        |   71 +-
 .../query/h2/GridH2ResultSetIterator.java       |  191 --
 .../query/h2/H2ConnectionWrapper.java           |   67 +
 .../processors/query/h2/H2DatabaseType.java     |  161 ++
 .../processors/query/h2/H2FieldsIterator.java   |   50 +
 .../processors/query/h2/H2KeyValueIterator.java |   48 +
 .../query/h2/H2ResultSetIterator.java           |  191 ++
 .../processors/query/h2/H2RowDescriptor.java    |  479 ++++
 .../internal/processors/query/h2/H2Schema.java  |  135 ++
 .../processors/query/h2/H2SqlFieldMetadata.java |  111 +
 .../processors/query/h2/H2StatementCache.java   |   73 +
 .../processors/query/h2/H2TableDescriptor.java  |  345 +++
 .../processors/query/h2/H2TableEngine.java      |   89 +
 .../query/h2/H2TwoStepCachedQuery.java          |   49 +
 .../query/h2/H2TwoStepCachedQueryKey.java       |  107 +
 .../internal/processors/query/h2/H2Utils.java   |  299 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 2112 +++---------------
 .../query/h2/database/H2PkHashIndex.java        |    4 +-
 .../query/h2/database/H2TreeIndex.java          |    4 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |   55 +-
 .../query/h2/opt/GridH2IndexBase.java           |   10 +-
 .../processors/query/h2/opt/GridH2Table.java    |    2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |    2 +-
 .../query/h2/opt/GridLuceneIndex.java           |   15 +-
 .../query/h2/sql/GridSqlCreateTable.java        |  153 ++
 .../query/h2/sql/GridSqlDropTable.java          |   79 +
 .../query/h2/sql/GridSqlQueryParser.java        |  260 +++
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    8 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   39 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   12 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |   18 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |    8 +-
 .../cache/index/H2DynamicTableSelfTest.java     |  320 +++
 .../query/IgniteQueryDedicatedPoolTest.java     |    6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  140 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  286 ++-
 .../IgniteCacheQuerySelfTestSuite.java          |    2 +
 .../cpp/odbc/include/ignite/odbc/message.h      |   10 +
 .../Impl/CacheFieldsQueryExecutor.cs            |   39 -
 .../Impl/CacheQueryableBase.cs                  |   14 +-
 .../Impl/ICacheQueryableInternal.cs             |    6 -
 .../org/apache/ignite/IgniteSpringBean.java     |    7 +
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../org/apache/ignite/yardstick/IgniteNode.java |    5 +
 .../apache/ignite/yardstick/io/FileUtils.java   |   64 +
 119 files changed, 8438 insertions(+), 4154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f651e87c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------


[03/39] ignite git commit: IGNITE-5163: Implemented infrastructure for the new JDBC driver. This closes #1912.

Posted by sb...@apache.org.
IGNITE-5163: Implemented infrastructure for the new JDBC driver. This closes #1912.


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 6f1dc3ac65d403a634331515cd1f279010d0d092
Parents: c04b39a
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Tue May 23 15:55:48 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue May 23 15:55:48 2017 +0300

----------------------------------------------------------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   3 +
 .../jdbc/thin/JdbcConnectionSelfTest.java       | 195 +++++++
 .../org/apache/ignite/IgniteJdbcThinDriver.java | 312 +++++++++++
 .../ignite/internal/GridKernalContext.java      |   8 +-
 .../ignite/internal/GridKernalContextImpl.java  |  12 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../internal/binary/BinaryWriterExImpl.java     |  16 +-
 .../internal/jdbc/thin/JdbcConnection.java      | 529 +++++++++++++++++++
 .../ignite/internal/jdbc/thin/JdbcTcpIo.java    | 207 ++++++++
 .../processors/odbc/OdbcNioListener.java        | 242 ---------
 .../internal/processors/odbc/OdbcProcessor.java | 199 -------
 .../odbc/SqlListenerAbstractMessageParser.java  | 265 ++++++++++
 .../odbc/SqlListenerAbstractObjectReader.java   | 137 +++++
 .../odbc/SqlListenerAbstractObjectWriter.java   | 111 ++++
 .../processors/odbc/SqlListenerNioListener.java | 263 +++++++++
 .../processors/odbc/SqlListenerProcessor.java   | 191 +++++++
 .../odbc/SqlListenerRequestHandlerImpl.java     | 494 +++++++++++++++++
 .../processors/odbc/jdbc/JdbcMessageParser.java |  50 ++
 .../processors/odbc/jdbc/JdbcObjectReader.java  |  33 ++
 .../processors/odbc/jdbc/JdbcObjectWriter.java  |  33 ++
 .../processors/odbc/odbc/OdbcMessageParser.java | 249 +--------
 .../processors/odbc/odbc/OdbcObjectReader.java  |  33 ++
 .../processors/odbc/odbc/OdbcObjectWriter.java  |  32 ++
 .../odbc/odbc/OdbcRequestHandler.java           | 513 ------------------
 .../odbc/OdbcProcessorValidationSelfTest.java   | 182 -------
 .../SqlListenerProcessorValidationSelfTest.java | 184 +++++++
 .../ignite/testframework/GridTestUtils.java     |   2 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   4 +-
 .../cpp/odbc/include/ignite/odbc/message.h      |  10 +
 29 files changed, 3124 insertions(+), 1389 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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 75671de..e2f09ba 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
@@ -81,6 +81,9 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDynamicIndexTransactionalPartitionedSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDynamicIndexTransactionalReplicatedSelfTest.class));
 
+        // New thin JDBC
+        suite.addTest(new TestSuite(org.apache.ignite.jdbc.thin.JdbcConnectionSelfTest.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcConnectionSelfTest.java
new file mode 100644
index 0000000..d7e2bef
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcConnectionSelfTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.concurrent.Callable;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.OdbcConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Connection test.
+ */
+public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** URL prefix. */
+    private static final String URL_PREFIX = "jdbc:ignite:thin://";
+
+    /** Host. */
+    private static final String HOST = "127.0.0.1";
+
+    /** {@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.setMarshaller(new BinaryMarshaller());
+
+        cfg.setOdbcConfiguration(new OdbcConfiguration());
+
+        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 {
+        try {
+            Driver drv = DriverManager.getDriver("jdbc:ignite://");
+
+            if (drv != null)
+                DriverManager.deregisterDriver(drv);
+        } catch (SQLException ignored) {
+            // No-op.
+        }
+
+        startGridsMultiThreaded(2);
+
+        Class.forName("org.apache.ignite.IgniteJdbcThinDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaults() throws Exception {
+        String url = URL_PREFIX + HOST;
+
+        assert DriverManager.getConnection(url) != null;
+        assert DriverManager.getConnection(url + "/") != null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvalidUrls() throws Exception {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                DriverManager.getConnection(URL_PREFIX + "127.0.0.1:80");
+
+                return null;
+            }
+        }, SQLException.class, "Failed to connect to Ignite cluster [host=127.0.0.1, port=80]");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                DriverManager.getConnection("q");
+
+                return null;
+            }
+        }, SQLException.class, "URL is invalid");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                DriverManager.getConnection(URL_PREFIX + "127.0.0.1:-1");
+
+                return null;
+            }
+        }, SQLException.class, "Invalid port:");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                DriverManager.getConnection(URL_PREFIX + "127.0.0.1:0");
+
+                return null;
+            }
+        }, SQLException.class, "Invalid port:");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                DriverManager.getConnection(URL_PREFIX + "127.0.0.1:100000");
+
+                return null;
+            }
+        }, SQLException.class, "Invalid port:");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                DriverManager.getConnection(URL_PREFIX + "     :10000");
+
+                return null;
+            }
+        }, SQLException.class, "Host name is empty");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClose() throws Exception {
+        String url = URL_PREFIX + HOST;
+
+        final Connection conn = DriverManager.getConnection(url);
+
+        assert conn != null;
+        assert !conn.isClosed();
+
+        conn.close();
+
+        assert conn.isClosed();
+
+        assert !conn.isValid(2): "Connection must be closed";
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    conn.isValid(-2);
+
+                    return null;
+                }
+            },
+            SQLException.class,
+            "Invalid timeout"
+        );
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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
new file mode 100644
index 0000000..19e1edd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
@@ -0,0 +1,312 @@
+/*
+ * 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;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.logging.Logger;
+import org.apache.ignite.cache.affinity.AffinityKey;
+import org.apache.ignite.configuration.OdbcConfiguration;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.jdbc.JdbcDriverPropertyInfo;
+import org.apache.ignite.internal.jdbc.thin.JdbcConnection;
+
+/**
+ * JDBC driver thin implementation for In-Memory Data Grid.
+ * <p>
+ * Driver allows to get distributed data from Ignite cache using standard
+ * SQL queries and standard JDBC API. It will automatically get only fields that
+ * you actually need from objects stored in cache.
+ * <h1 class="header">Limitations</h1>
+ * Data in Ignite cache is usually distributed across several nodes,
+ * so some queries may not work as expected since the query will be sent to each
+ * individual node and results will then be collected and returned as JDBC result set.
+ * Keep in mind following limitations (not applied if data is queried from one node only,
+ * or data is fully co-located or fully replicated on multiple nodes):
+ * <ul>
+ *     <li>
+ *         Joins will work correctly only if joined objects are stored in
+ *         collocated mode. Refer to
+ *         {@link AffinityKey}
+ *         javadoc for more details.
+ *     </li>
+ *     <li>
+ *         Note that if you are connected to local or replicated cache, all data will
+ *         be queried only on one node, not depending on what caches participate in
+ *         the query (some data from partitioned cache can be lost). And visa versa,
+ *         if you are connected to partitioned cache, data from replicated caches
+ *         will be duplicated.
+ *     </li>
+ * </ul>
+ * <h1 class="header">SQL Notice</h1>
+ * Driver allows to query data from several caches. Cache that driver is connected to is
+ * treated as default schema in this case. Other caches can be referenced by their names.\
+ *
+ * <h1 class="header">Dependencies</h1>
+ * JDBC driver is located in main Ignite JAR in {@code IGNITE_HOME/libs} folder.
+ * <h1 class="header">Configuration</h1>
+ *
+ * <p>
+ * JDBC connection URL has the following pattern:
+ * {@code jdbc:ignite://<hostname>:<port>/}<br>
+ * Note the following:
+ * <ul>
+ *     <li>Hostname is required.</li>
+ *     <li>If port is not defined, {@code 10800} is used (default for Ignite thin client).</li>
+ * </ul>
+ * Other properties can be defined in {@link Properties} object passed to
+ * {@link DriverManager#getConnection(String, Properties)} method:
+ * <table class="doctable">
+ *     <tr>
+ *         <th>Name</th>
+ *         <th>Description</th>
+ *         <th>Default</th>
+ *         <th>Optional</th>
+ *     </tr>
+ *     <tr>
+ *         <td><b>ignite.jdbc.distributedJoins</b></td>
+ *         <td>Flag to enable distributed joins.</td>
+ *         <td>{@code false} (distributed joins are disabled)</td>
+ *         <td>Yes</td>
+ *     </tr>
+ *     <tr>
+ *         <td><b>ignite.jdbc.enforceJoinOrder</b></td>
+ *         <td>Flag to enforce join order of tables in the query.</td>
+ *         <td>{@code false} (enforcing join order is disabled)</td>
+ *         <td>Yes</td>
+ *     </tr>
+ * </table>
+ * <h1 class="header">Example</h1>
+ * <pre name="code" class="java">
+ * // Register JDBC driver.
+ * Class.forName("org.apache.ignite.IgniteJdbcThinDriver");
+ *
+ * // Open JDBC connection.
+ * Connection conn = DriverManager.getConnection("jdbc:ignite:thin//localhost:10800");
+ *
+ * // Query persons' names
+ * ResultSet rs = conn.createStatement().executeQuery("select name from Person");
+ *
+ * while (rs.next()) {
+ *     String name = rs.getString(1);
+ *
+ *     ...
+ * }
+ *
+ * // Query persons with specific age
+ * PreparedStatement stmt = conn.prepareStatement("select name, age from Person where age = ?");
+ *
+ * stmt.setInt(1, 30);
+ *
+ * ResultSet rs = stmt.executeQuery();
+ *
+ * while (rs.next()) {
+ *     String name = rs.getString("name");
+ *     int age = rs.getInt("age");
+ *
+ *     ...
+ * }
+ * </pre>
+ */
+@SuppressWarnings("JavadocReference")
+public class IgniteJdbcThinDriver implements Driver {
+    /** Prefix for property names. */
+    private static final String PROP_PREFIX = "ignite.jdbc";
+
+    /** Distributed joins parameter name. */
+    private static final String PARAM_DISTRIBUTED_JOINS = "distributedJoins";
+
+    /** Enforce join order parameter name. */
+    private static final String ENFORCE_JOIN_ORDER = "enforceJoinOrder";
+
+    /** Hostname property name. */
+    public static final String PROP_HOST = PROP_PREFIX + "host";
+
+    /** Port number property name. */
+    public static final String PROP_PORT = PROP_PREFIX + "port";
+
+    /** Distributed joins property name. */
+    public static final String PROP_DISTRIBUTED_JOINS = PROP_PREFIX + PARAM_DISTRIBUTED_JOINS;
+
+    /** Transactions allowed property name. */
+    public static final String PROP_ENFORCE_JOIN_ORDER = PROP_PREFIX + ENFORCE_JOIN_ORDER;
+
+    /** URL prefix. */
+    public static final String URL_PREFIX = "jdbc:ignite:thin://";
+
+    /** Default port. */
+    public static final int DFLT_PORT = OdbcConfiguration.DFLT_TCP_PORT_FROM;
+
+    /** Major version. */
+    private static final int MAJOR_VER = IgniteVersionUtils.VER.major();
+
+    /** Minor version. */
+    private static final int MINOR_VER = IgniteVersionUtils.VER.minor();
+
+    /*
+     * Register driver.
+     */
+    static {
+        try {
+            DriverManager.registerDriver(new IgniteJdbcThinDriver());
+        }
+        catch (SQLException e) {
+            throw new RuntimeException("Failed to register Ignite JDBC driver.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Connection connect(String url, Properties props) throws SQLException {
+        if (!parseUrl(url, props))
+            throw new SQLException("URL is invalid: " + url);
+
+        return new JdbcConnection(url, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean acceptsURL(String url) throws SQLException {
+        return url.startsWith(URL_PREFIX);
+    }
+
+    /** {@inheritDoc} */
+    @Override public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException {
+        if (!parseUrl(url, info))
+            throw new SQLException("URL is invalid: " + url);
+
+        List<DriverPropertyInfo> props = Arrays.<DriverPropertyInfo>asList(
+            new JdbcDriverPropertyInfo("Hostname", info.getProperty(PROP_HOST), ""),
+            new JdbcDriverPropertyInfo("Port number", info.getProperty(PROP_PORT), ""),
+            new JdbcDriverPropertyInfo("Distributed Joins", info.getProperty(PROP_DISTRIBUTED_JOINS), ""),
+            new JdbcDriverPropertyInfo("Enforce Join Order", info.getProperty(PROP_ENFORCE_JOIN_ORDER), "")
+        );
+
+        return props.toArray(new DriverPropertyInfo[0]);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMajorVersion() {
+        return MAJOR_VER;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMinorVersion() {
+        return MINOR_VER;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean jdbcCompliant() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+        throw new SQLFeatureNotSupportedException("java.util.logging is not used.");
+    }
+
+    /**
+     * Validates and parses connection URL.
+     *
+     * @param props Properties.
+     * @param url URL.
+     * @return Whether URL is valid.
+     */
+    private boolean parseUrl(String url, Properties props) {
+        if (url == null)
+            return false;
+
+        if (url.startsWith(URL_PREFIX) && url.length() > URL_PREFIX.length())
+            return parseJdbcUrl(url, props);
+
+        return false;
+    }
+
+    /**
+     * @param url Url.
+     * @param props Properties.
+     * @return Whether URL is valid.
+     */
+    private boolean parseJdbcUrl(String url, Properties props) {
+        url = url.substring(URL_PREFIX.length());
+
+        String[] parts = url.split("\\?");
+
+        if (parts.length > 2)
+            return false;
+
+        if (parts.length == 2)
+            if (!parseParameters(parts[1], "&", props))
+                return false;
+
+        parts = parts[0].split("/");
+
+        assert parts.length > 0;
+
+        if (parts.length > 1)
+            return false;
+
+        url = parts[0];
+
+        parts = url.split(":");
+
+        assert parts.length > 0;
+
+        if (parts.length > 2)
+            return false;
+
+        props.setProperty(PROP_HOST, parts[0]);
+
+        try {
+            props.setProperty(PROP_PORT, String.valueOf(parts.length == 2 ? Integer.valueOf(parts[1]) : DFLT_PORT));
+        }
+        catch (NumberFormatException ignored) {
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Validates and parses URL parameters.
+     *
+     * @param val Parameters string.
+     * @param delim Delimiter.
+     * @param props Properties.
+     * @return Whether URL parameters string is valid.
+     */
+    private boolean parseParameters(String val, String delim, Properties props) {
+        String[] params = val.split(delim);
+
+        for (String param : params) {
+            String[] pair = param.split("=");
+
+            if (pair.length != 2 || pair[0].isEmpty() || pair[1].isEmpty())
+                return false;
+
+            props.setProperty(PROP_PREFIX + pair[0], pair[1]);
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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 010bd21..7a01200 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
@@ -48,7 +48,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
-import org.apache.ignite.internal.processors.odbc.OdbcProcessor;
+import org.apache.ignite.internal.processors.odbc.SqlListenerProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.processors.pool.PoolProcessor;
@@ -330,11 +330,11 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridQueryProcessor query();
 
     /**
-     * Gets ODBC processor.
+     * Gets SQL listener processor.
      *
-     * @return ODBC processor.
+     * @return SQL listener processor.
      */
-    public OdbcProcessor odbc();
+    public SqlListenerProcessor sqlListener();
 
     /**
      * @return Plugin processor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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 bbc9846..262c5eb 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
@@ -64,7 +64,7 @@ import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor;
-import org.apache.ignite.internal.processors.odbc.OdbcProcessor;
+import org.apache.ignite.internal.processors.odbc.SqlListenerProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.plugin.PlatformPluginProcessor;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
@@ -160,7 +160,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringInclude
-    private OdbcProcessor odbcProc;
+    private SqlListenerProcessor sqlListenerProc;
 
     /** */
     @GridToStringInclude
@@ -567,8 +567,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             pluginProc = (IgnitePluginProcessor)comp;
         else if (comp instanceof GridQueryProcessor)
             qryProc = (GridQueryProcessor)comp;
-        else if (comp instanceof OdbcProcessor)
-            odbcProc = (OdbcProcessor)comp;
+        else if (comp instanceof SqlListenerProcessor)
+            sqlListenerProc = (SqlListenerProcessor)comp;
         else if (comp instanceof DataStructuresProcessor)
             dataStructuresProc = (DataStructuresProcessor)comp;
         else if (comp instanceof ClusterProcessor)
@@ -824,8 +824,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public OdbcProcessor odbc() {
-        return odbcProc;
+    @Override public SqlListenerProcessor sqlListener() {
+        return sqlListenerProc;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/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 40476a7..c36fd7e 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
@@ -125,7 +125,7 @@ import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.OsDiscoveryNodeValidationProcessor;
-import org.apache.ignite.internal.processors.odbc.OdbcProcessor;
+import org.apache.ignite.internal.processors.odbc.SqlListenerProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformNoopProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.plugin.PlatformPluginProcessor;
@@ -924,7 +924,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx));
             startProcessor(new GridCacheProcessor(ctx));startProcessor(new GridClusterStateProcessor(ctx));
             startProcessor(new GridQueryProcessor(ctx));
-            startProcessor(new OdbcProcessor(ctx));
+            startProcessor(new SqlListenerProcessor(ctx));
             startProcessor(new GridServiceProcessor(ctx));
             startProcessor(new GridTaskSessionProcessor(ctx));
             startProcessor(new GridJobProcessor(ctx));

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
index 7b5e9d3..7efe4b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
@@ -938,7 +938,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeByteFieldPrimitive(byte val) {
+    public void writeByteFieldPrimitive(byte val) {
         out.unsafeEnsure(1 + 1);
 
         out.unsafeWriteByte(GridBinaryMarshaller.BYTE);
@@ -965,7 +965,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeShortFieldPrimitive(short val) {
+    public void writeShortFieldPrimitive(short val) {
         out.unsafeEnsure(1 + 2);
 
         out.unsafeWriteByte(GridBinaryMarshaller.SHORT);
@@ -985,7 +985,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeIntFieldPrimitive(int val) {
+    public void writeIntFieldPrimitive(int val) {
         out.unsafeEnsure(1 + 4);
 
         out.unsafeWriteByte(GridBinaryMarshaller.INT);
@@ -1005,7 +1005,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeLongFieldPrimitive(long val) {
+    public void writeLongFieldPrimitive(long val) {
         out.unsafeEnsure(1 + 8);
 
         out.unsafeWriteByte(GridBinaryMarshaller.LONG);
@@ -1025,7 +1025,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeFloatFieldPrimitive(float val) {
+    public void writeFloatFieldPrimitive(float val) {
         out.unsafeEnsure(1 + 4);
 
         out.unsafeWriteByte(GridBinaryMarshaller.FLOAT);
@@ -1045,7 +1045,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeDoubleFieldPrimitive(double val) {
+    public void writeDoubleFieldPrimitive(double val) {
         out.unsafeEnsure(1 + 8);
 
         out.unsafeWriteByte(GridBinaryMarshaller.DOUBLE);
@@ -1065,7 +1065,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeCharFieldPrimitive(char val) {
+    public void writeCharFieldPrimitive(char val) {
         out.unsafeEnsure(1 + 2);
 
         out.unsafeWriteByte(GridBinaryMarshaller.CHAR);
@@ -1085,7 +1085,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
-    void writeBooleanFieldPrimitive(boolean val) {
+    public void writeBooleanFieldPrimitive(boolean val) {
         out.unsafeEnsure(1 + 1);
 
         out.unsafeWriteByte(GridBinaryMarshaller.BOOLEAN);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcConnection.java
new file mode 100644
index 0000000..25d62b4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcConnection.java
@@ -0,0 +1,529 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executor;
+import java.util.logging.Logger;
+
+import static java.sql.ResultSet.CONCUR_READ_ONLY;
+import static java.sql.ResultSet.HOLD_CURSORS_OVER_COMMIT;
+import static java.sql.ResultSet.TYPE_FORWARD_ONLY;
+import static org.apache.ignite.IgniteJdbcThinDriver.PROP_DISTRIBUTED_JOINS;
+import static org.apache.ignite.IgniteJdbcThinDriver.PROP_ENFORCE_JOIN_ORDER;
+import static org.apache.ignite.IgniteJdbcThinDriver.PROP_HOST;
+import static org.apache.ignite.IgniteJdbcThinDriver.PROP_PORT;
+
+/**
+ * JDBC connection implementation.
+ *
+ * See documentation of {@link org.apache.ignite.IgniteJdbcThinDriver} for details.
+ */
+public class JdbcConnection implements Connection {
+    /** Logger. */
+    private static final Logger LOG = Logger.getLogger(JdbcConnection.class.getName());
+
+    /** Cache name. */
+    private String schemaName;
+
+    /** Closed flag. */
+    private boolean closed;
+
+    /** Current transaction isolation. */
+    private int txIsolation;
+
+    /** Auto commit flag. */
+    private boolean autoCommit;
+
+    /** Current transaction holdability. */
+    private int holdability;
+
+    /** Timeout. */
+    private int timeout;
+
+    /** Ignite endpoint. */
+    private JdbcTcpIo cliIo;
+
+    /**
+     * Creates new connection.
+     *
+     * @param url Connection URL.
+     * @param props Additional properties.
+     * @throws SQLException In case Ignite client failed to start.
+     */
+    public JdbcConnection(String url, Properties props) throws SQLException {
+        assert url != null;
+        assert props != null;
+
+        holdability = HOLD_CURSORS_OVER_COMMIT;
+        autoCommit = true;
+        txIsolation = Connection.TRANSACTION_NONE;
+
+        boolean distributedJoins = Boolean.parseBoolean(props.getProperty(PROP_DISTRIBUTED_JOINS, "false"));
+        boolean enforceJoinOrder = Boolean.parseBoolean(props.getProperty(PROP_ENFORCE_JOIN_ORDER, "false"));
+
+        String host = props.getProperty(PROP_HOST);
+        String portStr = props.getProperty(PROP_PORT);
+
+        try {
+            int port = Integer.parseInt(portStr);
+
+            if (port <= 0 || port > 0xFFFF)
+                throw new SQLException("Invalid port: " + portStr);
+        }
+        catch (NumberFormatException e) {
+            throw new SQLException("Invalid port: " + portStr, e);
+        }
+
+        if (host == null || host.trim().isEmpty())
+            throw new SQLException("Host name is empty.");
+
+        String endpoint = host.trim() + ":" + portStr.trim();
+
+        try {
+            cliIo = new JdbcTcpIo(endpoint, distributedJoins, enforceJoinOrder);
+
+            cliIo.start();
+        }
+        catch (Exception e) {
+            cliIo.close();
+
+            throw new SQLException("Failed to connect to Ignite cluster [host=" + host + ", port=" + portStr + ']', e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Statement createStatement() throws SQLException {
+        return createStatement(TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Statement createStatement(int resSetType, int resSetConcurrency) throws SQLException {
+        return createStatement(resSetType, resSetConcurrency, HOLD_CURSORS_OVER_COMMIT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Statement createStatement(int resSetType, int resSetConcurrency,
+        int resSetHoldability) throws SQLException {
+        ensureNotClosed();
+
+        checkCursorOptions(resSetType, resSetConcurrency, resSetHoldability);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareStatement(String sql) throws SQLException {
+        return prepareStatement(sql, TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareStatement(String sql, int resSetType,
+        int resSetConcurrency) throws SQLException {
+        return prepareStatement(sql, resSetType, resSetConcurrency, HOLD_CURSORS_OVER_COMMIT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareStatement(String sql, int resSetType, int resSetConcurrency,
+        int resSetHoldability) throws SQLException {
+        ensureNotClosed();
+
+        checkCursorOptions(resSetType, resSetConcurrency, resSetHoldability);
+
+        return null;
+    }
+
+    /**
+     * @param resSetType Cursor option.
+     * @param resSetConcurrency Cursor option.
+     * @param resSetHoldability Cursor option.
+     * @throws SQLException If options unsupported.
+     */
+    private void checkCursorOptions(int resSetType, int resSetConcurrency,
+        int resSetHoldability) throws SQLException {
+        if (resSetType != TYPE_FORWARD_ONLY)
+            throw new SQLFeatureNotSupportedException("Invalid result set type (only forward is supported.)");
+
+        if (resSetConcurrency != CONCUR_READ_ONLY)
+            throw new SQLFeatureNotSupportedException("Invalid concurrency (updates are not supported).");
+
+        if (resSetHoldability != HOLD_CURSORS_OVER_COMMIT)
+            LOG.warning("Transactions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public CallableStatement prepareCall(String sql) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Callable functions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public CallableStatement prepareCall(String sql, int resSetType, int resSetConcurrency)
+        throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Callable functions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String nativeSQL(String sql) throws SQLException {
+        ensureNotClosed();
+
+        return sql;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setAutoCommit(boolean autoCommit) throws SQLException {
+        ensureNotClosed();
+
+        this.autoCommit = autoCommit;
+
+        if (!autoCommit)
+            LOG.warning("Transactions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean getAutoCommit() throws SQLException {
+        ensureNotClosed();
+
+        if (!autoCommit)
+            LOG.warning("Transactions are not supported.");
+
+        return autoCommit;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void commit() throws SQLException {
+        ensureNotClosed();
+
+        LOG.warning("Transactions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rollback() throws SQLException {
+        ensureNotClosed();
+
+        LOG.warning("Transactions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws SQLException {
+        if (closed)
+            return;
+
+        closed = true;
+
+        cliIo.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClosed() throws SQLException {
+        return closed;
+    }
+
+    /** {@inheritDoc} */
+    @Override public DatabaseMetaData getMetaData() throws SQLException {
+        ensureNotClosed();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setReadOnly(boolean readOnly) throws SQLException {
+        ensureNotClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isReadOnly() throws SQLException {
+        ensureNotClosed();
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setCatalog(String catalog) throws SQLException {
+        ensureNotClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCatalog() throws SQLException {
+        ensureNotClosed();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setTransactionIsolation(int level) throws SQLException {
+        ensureNotClosed();
+
+        LOG.warning("Transactions are not supported.");
+
+        txIsolation = level;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getTransactionIsolation() throws SQLException {
+        ensureNotClosed();
+
+        LOG.warning("Transactions are not supported.");
+
+        return txIsolation;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SQLWarning getWarnings() throws SQLException {
+        ensureNotClosed();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clearWarnings() throws SQLException {
+        ensureNotClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Class<?>> getTypeMap() throws SQLException {
+        throw new SQLFeatureNotSupportedException("Types mapping is not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Types mapping is not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setHoldability(int holdability) throws SQLException {
+        ensureNotClosed();
+
+        if (holdability != HOLD_CURSORS_OVER_COMMIT)
+            LOG.warning("Transactions are not supported.");
+
+        this.holdability = holdability;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getHoldability() throws SQLException {
+        ensureNotClosed();
+
+        return holdability;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Savepoint setSavepoint() throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Savepoints are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Savepoint setSavepoint(String name) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Savepoints are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rollback(Savepoint savepoint) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Savepoints are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Savepoints are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public CallableStatement prepareCall(String sql, int resSetType, int resSetConcurrency,
+        int resSetHoldability) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Callable functions are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareStatement(String sql, int[] colIndexes) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareStatement(String sql, String[] colNames) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("Auto generated keys are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Clob createClob() throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Blob createBlob() throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public NClob createNClob() throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public SQLXML createSQLXML() throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isValid(int timeout) throws SQLException {
+        if (timeout < 0)
+            throw new SQLException("Invalid timeout: " + timeout);
+
+        return !closed;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setClientInfo(String name, String val) throws SQLClientInfoException {
+        throw new UnsupportedOperationException("Client info is not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setClientInfo(Properties props) throws SQLClientInfoException {
+        throw new UnsupportedOperationException("Client info is not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getClientInfo(String name) throws SQLException {
+        ensureNotClosed();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Properties getClientInfo() throws SQLException {
+        ensureNotClosed();
+
+        return new Properties();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Struct createStruct(String typeName, Object[] attrs) throws SQLException {
+        ensureNotClosed();
+
+        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("Connection is not a wrapper for " + iface.getName());
+
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface != null && iface == Connection.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setSchema(String schema) throws SQLException {
+        schemaName = schema;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSchema() throws SQLException {
+        return schemaName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void abort(Executor executor) throws SQLException {
+        close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setNetworkTimeout(Executor executor, int ms) throws SQLException {
+        if (ms < 0)
+            throw new IllegalArgumentException("Timeout is below zero: " + ms);
+
+        timeout = ms;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getNetworkTimeout() throws SQLException {
+        return timeout;
+    }
+
+    /**
+     * Ensures that connection is not closed.
+     *
+     * @throws SQLException If connection is closed.
+     */
+    private void ensureNotClosed() throws SQLException {
+        if (closed)
+            throw new SQLException("Connection is closed.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcTcpIo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcTcpIo.java
new file mode 100644
index 0000000..4946b41
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcTcpIo.java
@@ -0,0 +1,207 @@
+/*
+ * 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.jdbc.thin;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.util.logging.Logger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
+import org.apache.ignite.internal.processors.odbc.SqlListenerProtocolVersion;
+import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerNioListener;
+import org.apache.ignite.internal.util.ipc.IpcEndpoint;
+import org.apache.ignite.internal.util.ipc.IpcEndpointFactory;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * JDBC IO layer implementation based on blocking IPC streams.
+ */
+public class JdbcTcpIo {
+    /** Current version. */
+    private static final SqlListenerProtocolVersion CURRENT_VER = SqlListenerProtocolVersion.create(2, 1, 0);
+
+    /** Initial output stream capacity. */
+    private static final int HANDSHAKE_MSG_SIZE = 10;
+
+    /** Logger. */
+    private static final Logger log = Logger.getLogger(JdbcTcpIo.class.getName());
+
+    /** Server endpoint address. */
+    private final String endpointAddr;
+
+    /** Endpoint. */
+    private IpcEndpoint endpoint;
+
+    /** Output stream. */
+    private BufferedOutputStream out;
+
+    /** Input stream. */
+    private BufferedInputStream in;
+
+    /** Distributed joins. */
+    private boolean distributedJoins;
+
+    /** Enforce join order. */
+    private boolean enforceJoinOrder;
+
+    /** Closed flag. */
+    private boolean closed;
+
+    /**
+     * @param endpointAddr Endpoint.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce join order flag.
+     */
+    JdbcTcpIo(String endpointAddr, boolean distributedJoins, boolean enforceJoinOrder) {
+        assert endpointAddr != null;
+
+        this.endpointAddr = endpointAddr;
+        this.distributedJoins = distributedJoins;
+        this.enforceJoinOrder= enforceJoinOrder;
+    }
+
+    /**
+     * @throws IgniteCheckedException On error.
+     * @throws IOException On IO error in handshake.
+     */
+    public void start() throws IgniteCheckedException, IOException {
+        endpoint = IpcEndpointFactory.connectEndpoint(endpointAddr, null);
+
+        out = new BufferedOutputStream(endpoint.outputStream());
+        in = new BufferedInputStream(endpoint.inputStream());
+
+        handshake();
+    }
+
+    /**
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public void handshake() throws IOException, IgniteCheckedException {
+        BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(HANDSHAKE_MSG_SIZE),
+            null, null);
+
+        writer.writeByte((byte)SqlListenerRequest.HANDSHAKE);
+
+        writer.writeShort(CURRENT_VER.major());
+        writer.writeShort(CURRENT_VER.minor());
+        writer.writeShort(CURRENT_VER.maintenance());
+
+        writer.writeByte(SqlListenerNioListener.JDBC_CLIENT);
+
+        writer.writeBoolean(distributedJoins);
+        writer.writeBoolean(enforceJoinOrder);
+
+        send(writer.array());
+
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, new BinaryHeapInputStream(read()),
+            null, null, false);
+
+        boolean accepted = reader.readBoolean();
+
+        if (accepted)
+            return;
+
+        short maj = reader.readShort();
+        short min = reader.readShort();
+        short maintenance = reader.readShort();
+
+        String err = reader.readString();
+
+        SqlListenerProtocolVersion ver = SqlListenerProtocolVersion.create(maj, min, maintenance);
+
+        throw new IgniteCheckedException("Handshake failed [driverProtocolVer=" + CURRENT_VER +
+            ", remoteNodeProtocolVer=" + ver + ", err=" + err + ']');
+    }
+
+    /**
+     * @param req ODBC request.
+     * @throws IOException On error.
+     */
+    private void send(byte[] req) throws IOException {
+        int size = req.length;
+
+        out.write(size & 0xFF);
+        out.write((size >> 8) & 0xFF);
+        out.write((size >> 16) & 0xFF);
+        out.write((size >> 24) & 0xFF);
+
+        out.write(req);
+
+        out.flush();
+    }
+
+    /**
+     * @return Bytes of a response from server.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    private byte[] read() throws IOException, IgniteCheckedException {
+        byte[] sizeBytes = read(4);
+
+        int msgSize  = (((0xFF & sizeBytes[3]) << 24) | ((0xFF & sizeBytes[2]) << 16)
+            | ((0xFF & sizeBytes[1]) << 8) + (0xFF & sizeBytes[0]));
+
+        return read(msgSize);
+    }
+
+    /**
+     * @param size Count of bytes to read from stream.
+     * @return Read bytes.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    private byte [] read(int size) throws IOException, IgniteCheckedException {
+        int off = 0;
+
+        byte[] data = new byte[size];
+
+        while (off != size) {
+            int res = in.read(data, off, size - off);
+
+            if (res == -1)
+                throw new IgniteCheckedException("Failed to read incoming message (not enough data).");
+
+            off += res;
+        }
+
+        return data;
+    }
+
+    /**
+     * Close the client IO.
+     */
+    public void close() {
+        if (closed)
+            return;
+
+        // Clean up resources.
+        U.closeQuiet(out);
+        U.closeQuiet(in);
+
+        if (endpoint != null)
+            endpoint.close();
+
+        closed = true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
deleted file mode 100644
index cdb3de3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.odbc;
-
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.binary.BinaryReaderExImpl;
-import org.apache.ignite.internal.binary.BinaryWriterExImpl;
-import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
-import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
-import org.apache.ignite.internal.binary.streams.BinaryInputStream;
-import org.apache.ignite.internal.processors.odbc.odbc.OdbcMessageParser;
-import org.apache.ignite.internal.processors.odbc.odbc.OdbcRequestHandler;
-import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
-import org.apache.ignite.internal.util.nio.GridNioSession;
-import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
-import org.jetbrains.annotations.Nullable;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * ODBC message listener.
- */
-public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
-    /** Current version. */
-    private static final SqlListenerProtocolVersion CURRENT_VER = SqlListenerProtocolVersion.create(2, 1, 0);
-
-    /** Supported versions. */
-    private static final Set<SqlListenerProtocolVersion> SUPPORTED_VERS = new HashSet<>();
-
-    /** Connection-related metadata key. */
-    private static final int CONN_CTX_META_KEY = GridNioSessionMetaKey.nextUniqueKey();
-
-    /** Request ID generator. */
-    private static final AtomicLong REQ_ID_GEN = new AtomicLong();
-
-    /** Busy lock. */
-    private final GridSpinBusyLock busyLock;
-
-    /** Kernal context. */
-    private final GridKernalContext ctx;
-
-    /** Maximum allowed cursors. */
-    private final int maxCursors;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    static {
-        SUPPORTED_VERS.add(CURRENT_VER);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param ctx Context.
-     * @param busyLock Shutdown busy lock.
-     * @param maxCursors Maximum allowed cursors.
-     */
-    public OdbcNioListener(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) {
-        this.ctx = ctx;
-        this.busyLock = busyLock;
-        this.maxCursors = maxCursors;
-
-        log = ctx.log(getClass());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onConnected(GridNioSession ses) {
-        if (log.isDebugEnabled())
-            log.debug("SQL client connected: " + ses.remoteAddress());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
-        if (log.isDebugEnabled()) {
-            if (e == null)
-                log.debug("SQL client disconnected: " + ses.remoteAddress());
-            else
-                log.debug("SQL client disconnected due to an error [addr=" + ses.remoteAddress() + ", err=" + e + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onMessage(GridNioSession ses, byte[] msg) {
-        assert msg != null;
-
-        SqlListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY);
-
-        if (connCtx == null) {
-            onHandshake(ses, msg);
-
-            return;
-        }
-
-        SqlListenerMessageParser parser = connCtx.parser();
-
-        SqlListenerRequest req;
-
-        try {
-            req = parser.decode(msg);
-        }
-        catch (Exception e) {
-            log.error("Failed to parse SQL client request [err=" + e + ']');
-
-            ses.close();
-
-            return;
-        }
-
-        assert req != null;
-
-        req.requestId(REQ_ID_GEN.incrementAndGet());
-
-        try {
-            long startTime = 0;
-
-            if (log.isDebugEnabled()) {
-                startTime = System.nanoTime();
-
-                log.debug("SQL client request received [reqId=" + req.requestId() + ", addr=" + ses.remoteAddress() +
-                    ", req=" + req + ']');
-            }
-
-            SqlListenerRequestHandler handler = connCtx.handler();
-
-            SqlListenerResponse resp = handler.handle(req);
-
-            if (log.isDebugEnabled()) {
-                long dur = (System.nanoTime() - startTime) / 1000;
-
-                log.debug("SQL client request processed [reqId=" + req.requestId() + ", dur(mcs)=" + dur  +
-                    ", resp=" + resp.status() + ']');
-            }
-
-            byte[] outMsg = parser.encode(resp);
-
-            ses.send(outMsg);
-        }
-        catch (Exception e) {
-            log.error("Failed to process SQL client request [reqId=" + req.requestId() + ", err=" + e + ']');
-
-            ses.send(parser.encode(new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString())));
-        }
-    }
-
-    /**
-     * Perform handshake.
-     *
-     * @param ses Session.
-     * @param msg Message bytes.
-     */
-    private void onHandshake(GridNioSession ses, byte[] msg) {
-        BinaryInputStream stream = new BinaryHeapInputStream(msg);
-
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null, true);
-
-        byte cmd = reader.readByte();
-
-        if (cmd != SqlListenerRequest.HANDSHAKE) {
-            log.error("Unexpected SQL client request (will close session): " + ses.remoteAddress());
-
-            ses.close();
-
-            return;
-        }
-
-        short verMajor = reader.readShort();
-        short verMinor = reader.readShort();
-        short verMaintenance = reader.readShort();
-
-        SqlListenerProtocolVersion ver = SqlListenerProtocolVersion.create(verMajor, verMinor, verMaintenance);
-
-        String errMsg = null;
-
-        if (SUPPORTED_VERS.contains(ver)) {
-            // Prepare context.
-            SqlListenerConnectionContext connCtx = prepareContext(ver, reader);
-
-            ses.addMeta(CONN_CTX_META_KEY, connCtx);
-        }
-        else {
-            log.warning("Unsupported version: " + ver.toString());
-
-            errMsg = "Unsupported version.";
-        }
-
-        // Send response.
-        BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(8), null, null);
-
-        if (errMsg == null) {
-            writer.writeBoolean(true);
-        }
-        else {
-            writer.writeBoolean(false);
-            writer.writeShort(CURRENT_VER.major());
-            writer.writeShort(CURRENT_VER.minor());
-            writer.writeShort(CURRENT_VER.maintenance());
-            writer.doWriteString(errMsg);
-        }
-
-        ses.send(writer.array());
-    }
-
-    /**
-     * Prepare context.
-     *
-     * @param ver Version.
-     * @param reader Reader.
-     * @return Context.
-     */
-    private SqlListenerConnectionContext prepareContext(SqlListenerProtocolVersion ver, BinaryReaderExImpl reader) {
-        // TODO: Switch between ODBC and JDBC.
-        boolean distributedJoins = reader.readBoolean();
-        boolean enforceJoinOrder = reader.readBoolean();
-
-        OdbcRequestHandler handler =
-            new OdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder);
-
-        OdbcMessageParser parser = new OdbcMessageParser(ctx);
-
-        return new SqlListenerConnectionContext(handler, parser);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
deleted file mode 100644
index 6b8b5a3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.odbc;
-
-import java.net.InetAddress;
-import java.nio.ByteOrder;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.OdbcConfiguration;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.HostAndPortRange;
-import org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter;
-import org.apache.ignite.internal.util.nio.GridNioCodecFilter;
-import org.apache.ignite.internal.util.nio.GridNioFilter;
-import org.apache.ignite.internal.util.nio.GridNioServer;
-import org.apache.ignite.internal.util.nio.GridNioSession;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.spi.IgnitePortProtocol;
-import org.apache.ignite.thread.IgniteThreadPoolExecutor;
-
-/**
- * ODBC processor.
- */
-public class OdbcProcessor extends GridProcessorAdapter {
-    /** Default number of selectors. */
-    private static final int DFLT_SELECTOR_CNT = Math.min(4, Runtime.getRuntime().availableProcessors());
-
-    /** Default TCP_NODELAY flag. */
-    private static final boolean DFLT_TCP_NODELAY = true;
-
-    /** Default TCP direct buffer flag. */
-    private static final boolean DFLT_TCP_DIRECT_BUF = false;
-
-    /** Busy lock. */
-    private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
-
-    /** ODBC TCP Server. */
-    private GridNioServer<byte[]> srv;
-
-    /** ODBC executor service. */
-    private ExecutorService odbcExecSvc;
-
-    /**
-     * @param ctx Kernal context.
-     */
-    public OdbcProcessor(GridKernalContext ctx) {
-        super(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start(boolean activeOnStart) throws IgniteCheckedException {
-        IgniteConfiguration cfg = ctx.config();
-
-        OdbcConfiguration odbcCfg = cfg.getOdbcConfiguration();
-
-        if (odbcCfg != null) {
-            try {
-                Marshaller marsh = cfg.getMarshaller();
-
-                if (marsh != null && !(marsh instanceof BinaryMarshaller))
-                    throw new IgniteCheckedException("ODBC can only be used with BinaryMarshaller (please set it " +
-                        "through IgniteConfiguration.setMarshaller())");
-
-                HostAndPortRange hostPort;
-
-                if (F.isEmpty(odbcCfg.getEndpointAddress())) {
-                    hostPort = new HostAndPortRange(OdbcConfiguration.DFLT_TCP_HOST,
-                        OdbcConfiguration.DFLT_TCP_PORT_FROM,
-                        OdbcConfiguration.DFLT_TCP_PORT_TO
-                    );
-                }
-                else {
-                    hostPort = HostAndPortRange.parse(odbcCfg.getEndpointAddress(),
-                        OdbcConfiguration.DFLT_TCP_PORT_FROM,
-                        OdbcConfiguration.DFLT_TCP_PORT_TO,
-                        "Failed to parse ODBC endpoint address"
-                    );
-                }
-
-                assertParameter(odbcCfg.getThreadPoolSize() > 0, "threadPoolSize > 0");
-
-                odbcExecSvc = new IgniteThreadPoolExecutor(
-                    "odbc",
-                    cfg.getIgniteInstanceName(),
-                    odbcCfg.getThreadPoolSize(),
-                    odbcCfg.getThreadPoolSize(),
-                    0,
-                    new LinkedBlockingQueue<Runnable>());
-
-                InetAddress host;
-
-                try {
-                    host = InetAddress.getByName(hostPort.host());
-                }
-                catch (Exception e) {
-                    throw new IgniteCheckedException("Failed to resolve ODBC host: " + hostPort.host(), e);
-                }
-
-                Exception lastErr = null;
-
-                for (int port = hostPort.portFrom(); port <= hostPort.portTo(); port++) {
-                    try {
-                        GridNioFilter[] filters = new GridNioFilter[] {
-                            new GridNioAsyncNotifyFilter(ctx.igniteInstanceName(), odbcExecSvc, log) {
-                                @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
-                                    proceedSessionOpened(ses);
-                                }
-                            },
-                            new GridNioCodecFilter(new SqlListenerBufferedParser(), log, false)
-                        };
-
-                        GridNioServer<byte[]> srv0 = GridNioServer.<byte[]>builder()
-                            .address(host)
-                            .port(port)
-                            .listener(new OdbcNioListener(ctx, busyLock, odbcCfg.getMaxOpenCursors()))
-                            .logger(log)
-                            .selectorCount(DFLT_SELECTOR_CNT)
-                            .igniteInstanceName(ctx.igniteInstanceName())
-                            .serverName("odbc")
-                            .tcpNoDelay(DFLT_TCP_NODELAY)
-                            .directBuffer(DFLT_TCP_DIRECT_BUF)
-                            .byteOrder(ByteOrder.nativeOrder())
-                            .socketSendBufferSize(odbcCfg.getSocketSendBufferSize())
-                            .socketReceiveBufferSize(odbcCfg.getSocketReceiveBufferSize())
-                            .filters(filters)
-                            .directMode(false)
-                            .build();
-
-                        srv0.start();
-
-                        srv = srv0;
-
-                        ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass());
-
-                        log.info("ODBC processor has started on TCP port " + port);
-
-                        lastErr = null;
-
-                        break;
-                    }
-                    catch (Exception e) {
-                        lastErr = e;
-                    }
-                }
-
-                assert (srv != null && lastErr == null) || (srv == null && lastErr != null);
-
-                if (lastErr != null)
-                    throw new IgniteCheckedException("Failed to bind to any [host:port] from the range [" +
-                        "address=" + hostPort + ", lastErr=" + lastErr + ']');
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to start ODBC processor.", e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStop(boolean cancel) {
-        if (srv != null) {
-            busyLock.block();
-
-            srv.stop();
-
-            ctx.ports().deregisterPorts(getClass());
-
-            if (odbcExecSvc != null) {
-                U.shutdownNow(getClass(), odbcExecSvc, log);
-
-                odbcExecSvc = null;
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("ODBC processor stopped.");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractMessageParser.java
new file mode 100644
index 0000000..9d731ab
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractMessageParser.java
@@ -0,0 +1,265 @@
+/*
+ * 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.odbc;
+
+import java.util.Collection;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+
+/**
+ * ODBC message parser.
+ */
+public abstract class SqlListenerAbstractMessageParser implements SqlListenerMessageParser {
+    /** Initial output stream capacity. */
+    protected static final int INIT_CAP = 1024;
+
+    /** Kernal context. */
+    protected GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Object reader. */
+    private SqlListenerAbstractObjectReader objReader;
+
+    /** Object writer. */
+    private SqlListenerAbstractObjectWriter objWriter;
+
+    /**
+     * @param ctx Context.
+     * @param objReader Object reader.
+     * @param objWriter Object writer.
+     */
+    protected SqlListenerAbstractMessageParser(final GridKernalContext ctx, SqlListenerAbstractObjectReader objReader,
+        SqlListenerAbstractObjectWriter objWriter) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        this.objReader = objReader;
+        this.objWriter = objWriter;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlListenerRequest decode(byte[] msg) {
+        assert msg != null;
+
+        BinaryReaderExImpl reader = createReader(msg);
+
+        byte cmd = reader.readByte();
+
+        SqlListenerRequest res;
+
+        switch (cmd) {
+            case SqlListenerRequest.QRY_EXEC: {
+                String cache = reader.readString();
+                String sql = reader.readString();
+                int argsNum = reader.readInt();
+
+                Object[] params = new Object[argsNum];
+
+                for (int i = 0; i < argsNum; ++i)
+                    params[i] = objReader.readObject(reader);
+
+                res = new SqlListenerQueryExecuteRequest(cache, sql, params);
+
+                break;
+            }
+
+            case SqlListenerRequest.QRY_FETCH: {
+                long queryId = reader.readLong();
+                int pageSize = reader.readInt();
+
+                res = new SqlListenerQueryFetchRequest(queryId, pageSize);
+
+                break;
+            }
+
+            case SqlListenerRequest.QRY_CLOSE: {
+                long queryId = reader.readLong();
+
+                res = new SqlListenerQueryCloseRequest(queryId);
+
+                break;
+            }
+
+            case SqlListenerRequest.META_COLS: {
+                String cache = reader.readString();
+                String table = reader.readString();
+                String column = reader.readString();
+
+                res = new OdbcQueryGetColumnsMetaRequest(cache, table, column);
+
+                break;
+            }
+
+            case SqlListenerRequest.META_TBLS: {
+                String catalog = reader.readString();
+                String schema = reader.readString();
+                String table = reader.readString();
+                String tableType = reader.readString();
+
+                res = new OdbcQueryGetTablesMetaRequest(catalog, schema, table, tableType);
+
+                break;
+            }
+
+            case SqlListenerRequest.META_PARAMS: {
+                String cacheName = reader.readString();
+                String sqlQuery = reader.readString();
+
+                res = new OdbcQueryGetParamsMetaRequest(cacheName, sqlQuery);
+
+                break;
+            }
+
+            default:
+                throw new IgniteException("Unknown ODBC command: [cmd=" + cmd + ']');
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] encode(SqlListenerResponse msg) {
+        assert msg != null;
+
+        // Creating new binary writer
+        BinaryWriterExImpl writer = createWriter(INIT_CAP);
+
+        // Writing status.
+        writer.writeByte((byte) msg.status());
+
+        if (msg.status() != SqlListenerResponse.STATUS_SUCCESS) {
+            writer.writeString(msg.error());
+
+            return writer.array();
+        }
+
+        Object res0 = msg.response();
+
+        if (res0 == null)
+            return writer.array();
+        else if (res0 instanceof SqlListenerQueryExecuteResult) {
+            SqlListenerQueryExecuteResult res = (SqlListenerQueryExecuteResult) res0;
+
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.getQueryId());
+
+            writer.writeLong(res.getQueryId());
+
+            Collection<SqlListenerColumnMeta> metas = res.getColumnsMetadata();
+
+            assert metas != null;
+
+            writer.writeInt(metas.size());
+
+            for (SqlListenerColumnMeta meta : metas)
+                meta.write(writer);
+        }
+        else if (res0 instanceof SqlListenerQueryFetchResult) {
+            SqlListenerQueryFetchResult res = (SqlListenerQueryFetchResult) res0;
+
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.queryId());
+
+            writer.writeLong(res.queryId());
+
+            Collection<?> items0 = res.items();
+
+            assert items0 != null;
+
+            writer.writeBoolean(res.last());
+
+            writer.writeInt(items0.size());
+
+            for (Object row0 : items0) {
+                if (row0 != null) {
+                    Collection<?> row = (Collection<?>)row0;
+
+                    writer.writeInt(row.size());
+
+                    for (Object obj : row)
+                        objWriter.writeObject(writer, obj);
+                }
+            }
+        }
+        else if (res0 instanceof SqlListenerQueryCloseResult) {
+            SqlListenerQueryCloseResult res = (SqlListenerQueryCloseResult) res0;
+
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.getQueryId());
+
+            writer.writeLong(res.getQueryId());
+        }
+        else if (res0 instanceof OdbcQueryGetColumnsMetaResult) {
+            OdbcQueryGetColumnsMetaResult res = (OdbcQueryGetColumnsMetaResult) res0;
+
+            Collection<SqlListenerColumnMeta> columnsMeta = res.meta();
+
+            assert columnsMeta != null;
+
+            writer.writeInt(columnsMeta.size());
+
+            for (SqlListenerColumnMeta columnMeta : columnsMeta)
+                columnMeta.write(writer);
+        }
+        else if (res0 instanceof OdbcQueryGetTablesMetaResult) {
+            OdbcQueryGetTablesMetaResult res = (OdbcQueryGetTablesMetaResult) res0;
+
+            Collection<OdbcTableMeta> tablesMeta = res.meta();
+
+            assert tablesMeta != null;
+
+            writer.writeInt(tablesMeta.size());
+
+            for (OdbcTableMeta tableMeta : tablesMeta)
+                tableMeta.writeBinary(writer);
+        }
+        else if (res0 instanceof OdbcQueryGetParamsMetaResult) {
+            OdbcQueryGetParamsMetaResult res = (OdbcQueryGetParamsMetaResult) res0;
+
+            byte[] typeIds = res.typeIds();
+
+            objWriter.writeObject(writer, typeIds);
+        }
+        else
+            assert false : "Should not reach here.";
+
+        return writer.array();
+    }
+
+    /**
+     * Create reader.
+     *
+     * @param msg Input message.
+     * @return Reader.
+     */
+    protected abstract BinaryReaderExImpl createReader(byte[] msg);
+
+    /**
+     * Create writer.
+     *
+     * @param cap Initial capacity.
+     * @return Binary writer instance.
+     */
+    protected abstract BinaryWriterExImpl createWriter(int cap);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f1dc3ac/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectReader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectReader.java
new file mode 100644
index 0000000..18162e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerAbstractObjectReader.java
@@ -0,0 +1,137 @@
+/*
+ * 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.odbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Binary reader with marshaling non-primitive and non-embedded objects with JDK marshaller.
+ */
+@SuppressWarnings("unchecked")
+public abstract class SqlListenerAbstractObjectReader {
+    /**
+     * @param reader Reader.
+     * @return Read object.
+     * @throws BinaryObjectException On error.
+     */
+    @Nullable public Object readObject(BinaryReaderExImpl reader) throws BinaryObjectException {
+        byte type = reader.readByte();
+
+        switch (type) {
+            case GridBinaryMarshaller.NULL:
+                return null;
+
+            case GridBinaryMarshaller.BOOLEAN:
+                return reader.readBoolean();
+
+            case GridBinaryMarshaller.BYTE:
+                return reader.readByte();
+
+            case GridBinaryMarshaller.CHAR:
+                return reader.readChar();
+
+            case GridBinaryMarshaller.SHORT:
+                return reader.readShort();
+
+            case GridBinaryMarshaller.INT:
+                return reader.readInt();
+
+            case GridBinaryMarshaller.LONG:
+                return reader.readLong();
+
+            case GridBinaryMarshaller.FLOAT:
+                return reader.readFloat();
+
+            case GridBinaryMarshaller.DOUBLE:
+                return reader.readDouble();
+
+            case GridBinaryMarshaller.STRING:
+                return BinaryUtils.doReadString(reader.in());
+
+            case GridBinaryMarshaller.DECIMAL:
+                return BinaryUtils.doReadDecimal(reader.in());
+
+            case GridBinaryMarshaller.UUID:
+                return BinaryUtils.doReadUuid(reader.in());
+
+            case GridBinaryMarshaller.TIME:
+                return BinaryUtils.doReadTime(reader.in());
+
+            case GridBinaryMarshaller.TIMESTAMP:
+                return BinaryUtils.doReadTimestamp(reader.in());
+
+            case GridBinaryMarshaller.DATE:
+                return BinaryUtils.doReadDate(reader.in());
+
+            case GridBinaryMarshaller.BOOLEAN_ARR:
+                return BinaryUtils.doReadBooleanArray(reader.in());
+
+            case GridBinaryMarshaller.BYTE_ARR:
+                return BinaryUtils.doReadByteArray(reader.in());
+
+            case GridBinaryMarshaller.CHAR_ARR:
+                return BinaryUtils.doReadCharArray(reader.in());
+
+            case GridBinaryMarshaller.SHORT_ARR:
+                return BinaryUtils.doReadShortArray(reader.in());
+
+            case GridBinaryMarshaller.INT_ARR:
+                return BinaryUtils.doReadIntArray(reader.in());
+
+            case GridBinaryMarshaller.FLOAT_ARR:
+                return BinaryUtils.doReadFloatArray(reader.in());
+
+            case GridBinaryMarshaller.DOUBLE_ARR:
+                return BinaryUtils.doReadDoubleArray(reader.in());
+
+            case GridBinaryMarshaller.STRING_ARR:
+                return BinaryUtils.doReadStringArray(reader.in());
+
+            case GridBinaryMarshaller.DECIMAL_ARR:
+                return BinaryUtils.doReadDecimalArray(reader.in());
+
+            case GridBinaryMarshaller.UUID_ARR:
+                return BinaryUtils.doReadUuidArray(reader.in());
+
+            case GridBinaryMarshaller.TIME_ARR:
+                return BinaryUtils.doReadTimeArray(reader.in());
+
+            case GridBinaryMarshaller.TIMESTAMP_ARR:
+                return BinaryUtils.doReadTimestampArray(reader.in());
+
+            case GridBinaryMarshaller.DATE_ARR:
+                return BinaryUtils.doReadDateArray(reader.in());
+
+            default:
+                reader.in().position(reader.in().position() - 1);
+
+                return readCustomObject(reader);
+        }
+    }
+
+    /**
+     * @param reader Reader.
+     * @return An object is unmarshaled by marshaller.
+     * @throws BinaryObjectException On error.
+     */
+    protected abstract Object readCustomObject(BinaryReaderExImpl reader) throws BinaryObjectException;
+}


[39/39] ignite git commit: Merge branch 'ignite-5075-cc' into ignite-5075-cc-debug

Posted by sb...@apache.org.
Merge branch 'ignite-5075-cc' into ignite-5075-cc-debug

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 0e069af54d3d2443f51475cf3f1974f24bb443fb
Parents: 1f3ceeb
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 17:05:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 17:05:50 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryFailoverAbstractSelfTest.java          | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e069af5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index e63e9bd..06dffb6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -840,7 +840,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
             IgniteCache<Object, Object> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
-            List<Integer> keys = testKeys(cache, PARTS);
+            List<Integer> keys = testKeys(cache, 1);
 
             boolean first = true;
 
@@ -869,6 +869,8 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                         expEvts.add(new T3<>((Object)key, (Object)val, (Object)key));
                 }
 
+                TestDebugLog.addEntryMessage(ignite(4).affinity(DEFAULT_CACHE_NAME).partition(key), filtered, "do put " + val);
+
                 cache.put(key, val);
 
                 if (first) {
@@ -1323,7 +1325,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
         int[] nodeParts = aff.primaryPartitions(node);
 
-        final int KEYS_PER_PART = 50;
+        final int KEYS_PER_PART = 2;
 
         for (int i = 0; i < parts; i++) {
             int part = nodeParts[i];


[35/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: 592f2a71592274b091f372e1ae228716947ab0f1
Parents: 6c73a36
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 14:14:16 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 14:14:16 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryEventBuffer.java          | 18 ++++++++++++------
 1 file changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/592f2a71/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
index 31b7ace..7d33614 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEventBuffer.java
@@ -23,12 +23,12 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentLinkedDeque8;
 
 /**
  *
@@ -48,7 +48,7 @@ public class CacheContinuousQueryEventBuffer {
     private AtomicReference<Batch> curBatch = new AtomicReference<>();
 
     /** */
-    private ConcurrentLinkedDeque<CacheContinuousQueryEntry> backupQ = new ConcurrentLinkedDeque<>();
+    private ConcurrentLinkedDeque8<CacheContinuousQueryEntry> backupQ = new ConcurrentLinkedDeque8<>();
 
     /** */
     private ConcurrentSkipListMap<Long, CacheContinuousQueryEntry> pending = new ConcurrentSkipListMap<>();
@@ -85,14 +85,20 @@ public class CacheContinuousQueryEventBuffer {
         if (batch != null)
             ret = batch.flushCurrentEntries();
 
-        if (!backupQ.isEmpty()) {
+        int size = backupQ.sizex();
+
+        if (size > 0) {
             if (ret == null)
                 ret = new ArrayList<>();
 
-            CacheContinuousQueryEntry e;
+            for (int i = 0; i < size; i++) {
+                CacheContinuousQueryEntry e = backupQ.pollFirst();
 
-            while ((e = backupQ.pollFirst()) != null)
-                ret.add(e);
+                if (e != null)
+                    ret.add(e);
+                else
+                    break;
+            }
         }
 
         if (!pending.isEmpty()) {


[36/39] ignite git commit: cc

Posted by sb...@apache.org.
cc


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: be43bf8fe254fe34f500e3395deb3cbabf6b0ea0
Parents: 592f2a7
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 26 14:55:37 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 26 14:55:37 2017 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java      | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be43bf8f/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 dfea951..72c5bbc 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
@@ -1101,7 +1101,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /**
      * @return {@code True} if exchange triggered by server node join or fail.
      */
-    private boolean serverNotDiscoveryEvent() {
+    private boolean serverNodeDiscoveryEvent() {
+        assert discoEvt != null;
+
         return discoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT && !CU.clientNode(discoEvt.eventNode());
     }
 
@@ -1109,7 +1111,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     @Override public boolean onDone(@Nullable AffinityTopologyVersion res, @Nullable Throwable err) {
         boolean realExchange = !dummy && !forcePreload;
 
-        if (realExchange && !cctx.kernalContext().clientNode() && (serverNotDiscoveryEvent() || affChangeMsg != null)) {
+        if (err == null &&
+            realExchange &&
+            !cctx.kernalContext().clientNode() &&
+            (serverNodeDiscoveryEvent() || affChangeMsg != null)) {
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
                 if (!cacheCtx.affinityNode() || cacheCtx.isLocal())
                     continue;


[21/39] ignite git commit: IGNITE-5052: DDL: Implemented CREATE TABLE and DROP TABLE commands. This closes #1958.

Posted by sb...@apache.org.
IGNITE-5052: DDL: Implemented CREATE TABLE and DROP TABLE commands. This closes #1958.


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

Branch: refs/heads/ignite-5075-cc-debug
Commit: e8f5af333a6f02f103a257058b2740f6a510a367
Parents: af026d1
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu May 25 13:48:40 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu May 25 13:48:40 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   5 +-
 .../org/apache/ignite/internal/IgniteEx.java    |  27 ++
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../processors/cache/ClusterCachesInfo.java     |  10 +-
 .../processors/cache/ExchangeActions.java       |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  43 ++-
 .../processors/query/GridQueryProcessor.java    |  52 +++
 .../handlers/cache/GridCacheCommandHandler.java |   8 +-
 .../processors/igfs/IgfsIgniteMock.java         |  17 +
 .../junits/multijvm/IgniteProcessProxy.java     |  13 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |  55 +++-
 .../query/h2/sql/GridSqlCreateTable.java        | 153 +++++++++
 .../query/h2/sql/GridSqlDropTable.java          |  79 +++++
 .../query/h2/sql/GridSqlQueryParser.java        | 260 +++++++++++++++
 .../cache/index/H2DynamicTableSelfTest.java     | 320 +++++++++++++++++++
 .../query/h2/sql/GridQueryParsingTest.java      | 284 +++++++++++++++-
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 17 files changed, 1326 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 11fc43a..1136c71 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -1794,9 +1794,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @return {@code this} for chaining.
      */
     public CacheConfiguration<K, V> setQueryEntities(Collection<QueryEntity> qryEntities) {
-        if (this.qryEntities == null)
+        if (this.qryEntities == null) {
             this.qryEntities = new ArrayList<>(qryEntities);
 
+            return this;
+        }
+
         for (QueryEntity entity : qryEntities) {
             boolean found = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index 421d6f9..8c27027 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -18,13 +18,17 @@
 package org.apache.ignite.internal;
 
 import java.util.Collection;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
@@ -60,6 +64,29 @@ public interface IgniteEx extends Ignite {
     public Collection<IgniteInternalCache<?, ?>> cachesx(@Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p);
 
     /**
+     * Gets existing cache with the given name or creates new one with the given configuration.
+     * <p>
+     * If a cache with the same name already exists, this method will not check that the given
+     * configuration matches the configuration of existing cache and will return an instance
+     * of the existing cache.
+     *
+     * @param cacheCfg Cache configuration to use.
+     * @return Tuple [Existing or newly created cache; {@code true} if cache was newly crated, {@code false} otherwise]
+     * @throws CacheException If error occurs.
+     */
+    public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(CacheConfiguration<K, V> cacheCfg)
+        throws CacheException;
+
+    /**
+     * Stops dynamically started cache.
+     *
+     * @param cacheName Cache name to stop.
+     * @return {@code true} if cache has been stopped as the result of this call, {@code false} otherwise.
+     * @throws CacheException If error occurs.
+     */
+    public boolean destroyCache0(String cacheName) throws CacheException;
+
+    /**
      * Checks if the event type is user-recordable.
      *
      * @param type Event type to check.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 c36fd7e..0f43169 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
@@ -46,6 +46,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.CacheException;
 import javax.management.JMException;
 import javax.management.ObjectName;
 import org.apache.ignite.IgniteAtomicLong;
@@ -163,6 +164,7 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
@@ -2789,6 +2791,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
+        return getOrCreateCache0(cacheCfg).get1();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
+        CacheConfiguration<K, V> cacheCfg) {
         A.notNull(cacheCfg, "cacheCfg");
         CU.validateCacheName(cacheCfg.getName());
 
@@ -2797,8 +2806,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             checkClusterState();
 
+            Boolean res = false;
+
             if (ctx.cache().cache(cacheCfg.getName()) == null) {
-                ctx.cache().dynamicStartCache(cacheCfg,
+                res = ctx.cache().dynamicStartCache(cacheCfg,
                     cacheCfg.getName(),
                     null,
                     false,
@@ -2806,7 +2817,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     true).get();
             }
 
-            return ctx.cache().publicJCache(cacheCfg.getName());
+            return new IgniteBiTuple<>((IgniteCache<K, V>)ctx.cache().publicJCache(cacheCfg.getName()), res);
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -3006,12 +3017,17 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
+        destroyCache0(cacheName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean destroyCache0(String cacheName) throws CacheException {
         CU.validateCacheName(cacheName);
 
-        IgniteInternalFuture stopFut = destroyCacheAsync(cacheName, true);
+        IgniteInternalFuture<Boolean> stopFut = destroyCacheAsync(cacheName, true);
 
         try {
-            stopFut.get();
+            return stopFut.get();
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -3037,7 +3053,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Ignite future.
      */
-    public IgniteInternalFuture<?> destroyCacheAsync(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> destroyCacheAsync(String cacheName, boolean checkThreadTx) {
         CU.validateCacheName(cacheName);
 
         guard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 7150cf4..ab5cf37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -272,7 +272,7 @@ class ClusterCachesInfo {
             if (req.start()) {
                 if (desc == null) {
                     if (req.clientStartOnly()) {
-                        ctx.cache().completeCacheStartFuture(req, new IgniteCheckedException("Failed to start " +
+                        ctx.cache().completeCacheStartFuture(req, false, new IgniteCheckedException("Failed to start " +
                             "client cache (a cache with the given name is not started): " + req.cacheName()));
                     }
                     else {
@@ -327,7 +327,7 @@ class ClusterCachesInfo {
                     }
                     else {
                         if (req.failIfExists()) {
-                            ctx.cache().completeCacheStartFuture(req,
+                            ctx.cache().completeCacheStartFuture(req, false,
                                 new CacheExistsException("Failed to start cache " +
                                     "(a cache with the same name is already started): " + req.cacheName()));
                         }
@@ -420,7 +420,7 @@ class ClusterCachesInfo {
         if (!F.isEmpty(reqsToComplete)) {
             ctx.closure().callLocalSafe(new Callable<Void>() {
                 @Override public Void call() throws Exception {
-                    for (T2<DynamicCacheChangeRequest, AffinityTopologyVersion> t :reqsToComplete) {
+                    for (T2<DynamicCacheChangeRequest, AffinityTopologyVersion> t : reqsToComplete) {
                         final DynamicCacheChangeRequest req = t.get1();
                         AffinityTopologyVersion waitTopVer = t.get2();
 
@@ -428,11 +428,11 @@ class ClusterCachesInfo {
                             ctx.cache().context().exchange().affinityReadyFuture(waitTopVer) : null;
 
                         if (fut == null || fut.isDone())
-                            ctx.cache().completeCacheStartFuture(req, null);
+                            ctx.cache().completeCacheStartFuture(req, false, null);
                         else {
                             fut.listen(new IgniteInClosure<IgniteInternalFuture<?>>() {
                                 @Override public void apply(IgniteInternalFuture<?> fut) {
-                                    ctx.cache().completeCacheStartFuture(req, null);
+                                    ctx.cache().completeCacheStartFuture(req, false, null);
                                 }
                             });
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index eac1120..5ac51ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -132,7 +132,7 @@ public class ExchangeActions {
     private void completeRequestFutures(Map<String, ActionData> map, GridCacheSharedContext ctx) {
         if (map != null) {
             for (ActionData req : map.values())
-                ctx.cache().completeCacheStartFuture(req.req, null);
+                ctx.cache().completeCacheStartFuture(req.req, true, null);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index a716ff4..d94c41f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1967,12 +1967,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param req Request to complete future for.
      * @param err Error if any.
      */
-    void completeCacheStartFuture(DynamicCacheChangeRequest req, @Nullable Exception err) {
+    void completeCacheStartFuture(DynamicCacheChangeRequest req, boolean success, @Nullable Exception err) {
         if (req.initiatingNodeId().equals(ctx.localNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
             if (fut != null)
-                fut.onDone(null, err);
+                fut.onDone(success, err);
         }
     }
 
@@ -2065,7 +2065,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     public IgniteInternalFuture<?> createFromTemplate(String cacheName) {
         try {
-            CacheConfiguration cfg = createConfigFromTemplate(cacheName);
+            CacheConfiguration cfg = getOrCreateConfigFromTemplate(cacheName);
 
             return dynamicStartCache(cfg, cacheName, null, true, true, true);
         }
@@ -2088,7 +2088,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (publicJCache(cacheName, false, checkThreadTx) != null) // Cache with given name already started.
                 return new GridFinishedFuture<>();
 
-            CacheConfiguration cfg = createConfigFromTemplate(cacheName);
+            CacheConfiguration cfg = getOrCreateConfigFromTemplate(cacheName);
 
             return dynamicStartCache(cfg, cacheName, null, false, true, checkThreadTx);
         }
@@ -2099,10 +2099,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cacheName Cache name.
-     * @return Cache configuration.
+     * @return Cache configuration, or {@code null} if no template with matching name found.
      * @throws IgniteCheckedException If failed.
      */
-    private CacheConfiguration createConfigFromTemplate(String cacheName) throws IgniteCheckedException {
+    public CacheConfiguration getConfigFromTemplate(String cacheName) throws IgniteCheckedException {
         CacheConfiguration cfgTemplate = null;
 
         CacheConfiguration dfltCacheCfg = null;
@@ -2160,7 +2160,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cfgTemplate == null)
             cfgTemplate = dfltCacheCfg;
 
-        cfgTemplate = cfgTemplate == null ? new CacheConfiguration() : cloneCheckSerializable(cfgTemplate);
+        if (cfgTemplate == null)
+            return null;
+
+        cfgTemplate = cloneCheckSerializable(cfgTemplate);
 
         CacheConfiguration cfg = new CacheConfiguration(cfgTemplate);
 
@@ -2170,6 +2173,20 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     * @throws IgniteCheckedException If failed.
+     */
+    private CacheConfiguration getOrCreateConfigFromTemplate(String cacheName) throws IgniteCheckedException {
+        CacheConfiguration cfg = getConfigFromTemplate(cacheName);
+
+        if (cfg != null)
+            return cfg;
+        else
+            return new CacheConfiguration(cacheName);
+    }
+
+    /**
      * Dynamically starts cache.
      *
      * @param ccfg Cache configuration.
@@ -2181,7 +2198,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when cache is deployed.
      */
     @SuppressWarnings("IfMayBeConditional")
-    public IgniteInternalFuture<?> dynamicStartCache(
+    public IgniteInternalFuture<Boolean> dynamicStartCache(
         @Nullable CacheConfiguration ccfg,
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
@@ -2211,7 +2228,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when cache is deployed.
      */
     @SuppressWarnings("IfMayBeConditional")
-    public IgniteInternalFuture<?> dynamicStartCache(
+    public IgniteInternalFuture<Boolean> dynamicStartCache(
         @Nullable CacheConfiguration ccfg,
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
@@ -2315,7 +2332,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Future that will be completed when cache is destroyed.
      */
-    public IgniteInternalFuture<?> dynamicDestroyCache(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> dynamicDestroyCache(String cacheName, boolean checkThreadTx) {
         assert cacheName != null;
 
         if (checkThreadTx)
@@ -2517,7 +2534,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     if (desc == null)
                         // No-op.
-                        fut.onDone();
+                        fut.onDone(false);
                     else {
                         assert desc.cacheConfiguration() != null : desc;
 
@@ -3452,7 +3469,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      *
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    private class DynamicCacheStartFuture extends GridFutureAdapter<Object> {
+    private class DynamicCacheStartFuture extends GridFutureAdapter<Boolean> {
         /** Cache name. */
         private String cacheName;
 
@@ -3477,7 +3494,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
+        @Override public boolean onDone(@Nullable Boolean res, @Nullable Throwable err) {
             // Make sure to remove future before completion.
             pendingFuts.remove(req.requestId(), this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 a40c9e9..09dbe42 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
@@ -1266,6 +1266,58 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Create cache and table from given query entity.
+     *
+     * @param schemaName Schema name to create table in.
+     * @param entity Entity to create table from.
+     * @param templateCacheName Cache name to take settings from.
+     * @param ifNotExists Quietly ignore this command if table already exists.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void dynamicTableCreate(String schemaName, QueryEntity entity, String templateCacheName, boolean ifNotExists)
+        throws IgniteCheckedException {
+        CacheConfiguration<?, ?> templateCfg = ctx.cache().getConfigFromTemplate(templateCacheName);
+
+        if (templateCfg == null)
+            throw new SchemaOperationException(SchemaOperationException.CODE_CACHE_NOT_FOUND, templateCacheName);
+
+        if (!F.isEmpty(templateCfg.getQueryEntities()))
+            throw new SchemaOperationException("Template cache already contains query entities which it should not " +
+                "[cacheName=" + templateCacheName + ']');
+
+        CacheConfiguration<?, ?> newCfg = new CacheConfiguration<>(templateCfg);
+
+        newCfg.setName(entity.getTableName());
+
+        newCfg.setQueryEntities(Collections.singleton(entity));
+
+        // We want to preserve user specified names as they are
+        newCfg.setSqlEscapeAll(true);
+
+        boolean res = ctx.grid().getOrCreateCache0(newCfg).get2();
+
+        if (!res && !ifNotExists)
+            throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_EXISTS,  entity.getTableName());
+    }
+
+    /**
+     * Drop table by destroying its cache if it's an 1:1 per cache table.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param ifExists Quietly ignore this command if table does not exist.
+     * @throws SchemaOperationException if {@code ifExists} is {@code false} and cache was not found.
+     */
+    @SuppressWarnings("unchecked")
+    public void dynamicTableDrop(String schemaName, String tblName, boolean ifExists) throws SchemaOperationException {
+        boolean res = ctx.grid().destroyCache0(tblName);
+
+        if (!res && !ifExists)
+            throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName);
+    }
+
+    /**
      * Register cache in indexing SPI.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 9edf37e..3b9fa69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -383,7 +383,9 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                         new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
                             @Override public GridRestResponse applyx(IgniteInternalFuture<?> f)
                                 throws IgniteCheckedException {
-                                return new GridRestResponse(f.get());
+                                f.get();
+
+                                return new GridRestResponse(null);
                             }
                         });
 
@@ -396,7 +398,9 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                         new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
                             @Override public GridRestResponse applyx(IgniteInternalFuture<?> f)
                                 throws IgniteCheckedException {
-                                return new GridRestResponse(f.get());
+                                f.get();
+
+                                return new GridRestResponse(null);
                             }
                         });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index ecba1bf..cc058b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.plugin.IgnitePlugin;
@@ -62,6 +63,7 @@ import org.jetbrains.annotations.Nullable;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.concurrent.ExecutorService;
+import javax.cache.CacheException;
 
 /**
  * Mocked Ignite implementation for IGFS tests.
@@ -320,6 +322,21 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
+        CacheConfiguration<K, V> cacheCfg) {
+        throwUnsupported();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+        throwUnsupported();
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
         throwUnsupported();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 26c86dc..34ca22f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteAtomicReference;
@@ -73,6 +74,7 @@ import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -533,11 +535,22 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
+        CacheConfiguration<K, V> cacheCfg) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
+    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void destroyCaches(Collection<String> cacheNames) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 3991777..7157d1f 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
@@ -23,6 +23,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.internal.GridKernalContext;
@@ -34,16 +35,23 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.h2.command.Prepared;
 import org.h2.command.ddl.CreateIndex;
+import org.h2.command.ddl.CreateTable;
 import org.h2.command.ddl.DropIndex;
+import org.h2.command.ddl.DropTable;
 import org.h2.jdbc.JdbcPreparedStatement;
+import org.h2.table.Column;
+import org.h2.value.DataType;
 
 import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
 
@@ -137,11 +145,27 @@ public class DdlStatementsProcessor {
                             dropIdx.indexName());
                 }
             }
+            else if (gridStmt instanceof GridSqlCreateTable) {
+                GridSqlCreateTable createTbl = (GridSqlCreateTable)gridStmt;
+
+                ctx.query().dynamicTableCreate(createTbl.schemaName(), toQueryEntity(createTbl),
+                    createTbl.templateCacheName(), createTbl.ifNotExists());
+
+                fut = null;
+            }
+            else if (gridStmt instanceof GridSqlDropTable) {
+                GridSqlDropTable dropTbl = (GridSqlDropTable)gridStmt;
+
+                ctx.query().dynamicTableDrop(dropTbl.schemaName(), dropTbl.tableName(), dropTbl.ifExists());
+
+                fut = null;
+            }
             else
                 throw new IgniteSQLException("Unsupported DDL operation: " + sql,
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
-            fut.get();
+            if (fut != null)
+                fut.get();
 
             QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
                 (Collections.singletonList(0L)), null, false);
@@ -211,10 +235,37 @@ public class DdlStatementsProcessor {
     }
 
     /**
+     * Convert this statement to query entity and do Ignite specific sanity checks on the way.
+     * @return Query entity mimicking this SQL statement.
+     */
+    private static QueryEntity toQueryEntity(GridSqlCreateTable createTbl) {
+        QueryEntity res = new QueryEntity();
+
+        res.setTableName(createTbl.tableName());
+
+        for (Map.Entry<String, GridSqlColumn> e : createTbl.columns().entrySet()) {
+            GridSqlColumn gridCol = e.getValue();
+
+            Column col = gridCol.column();
+
+            res.addQueryField(e.getKey(), DataType.getTypeClassName(col.getType()), null);
+        }
+
+        res.setKeyType(createTbl.tableName() + "Key");
+
+        res.setValueType(createTbl.tableName());
+
+        res.setKeyFields(createTbl.primaryKeyColumns());
+
+        return res;
+    }
+
+    /**
      * @param cmd Statement.
      * @return Whether {@code cmd} is a DDL statement we're able to handle.
      */
     public static boolean isDdlStatement(Prepared cmd) {
-        return cmd instanceof CreateIndex || cmd instanceof DropIndex;
+        return cmd instanceof CreateIndex || cmd instanceof DropIndex || cmd instanceof CreateTable ||
+            cmd instanceof DropTable;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
new file mode 100644
index 0000000..52c9cc9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.sql;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+
+/**
+ * CREATE TABLE statement.
+ */
+public class GridSqlCreateTable extends GridSqlStatement {
+    /**
+     * Schema name upon which this statement has been issued - <b>not</b> the name of the schema where this new table
+     * will be created. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Cache name upon which new cache configuration for this table must be based. */
+    private String tplCacheName;
+
+    /** Quietly ignore this command if table already exists. */
+    private boolean ifNotExists;
+
+    /** Columns. */
+    private LinkedHashMap<String, GridSqlColumn> cols;
+
+    /** Primary key columns. */
+    private LinkedHashSet<String> pkCols;
+
+    /** Extra WITH-params. */
+    private List<String> params;
+
+    /**
+     * @return Cache name upon which new cache configuration for this table must be based.
+     */
+    public String templateCacheName() {
+        return tplCacheName;
+    }
+
+    /**
+     * @param tplCacheName Cache name upon which new cache configuration for this table must be based.
+     */
+    public void templateCacheName(String tplCacheName) {
+        this.tplCacheName = tplCacheName;
+    }
+
+    /**
+     * @return Columns.
+     */
+    public LinkedHashMap<String, GridSqlColumn> columns() {
+        return cols;
+    }
+
+    /**
+     * @param cols Columns.
+     */
+    public void columns(LinkedHashMap<String, GridSqlColumn> cols) {
+        this.cols = cols;
+    }
+
+    /**
+     * @return Primary key columns.
+     */
+    public LinkedHashSet<String> primaryKeyColumns() {
+        return pkCols;
+    }
+
+    /**
+     * @param pkCols Primary key columns.
+     */
+    public void primaryKeyColumns(LinkedHashSet<String> pkCols) {
+        this.pkCols = pkCols;
+    }
+
+    /**
+     * @return Schema name upon which this statement has been issued.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name upon which this statement has been issued.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Quietly ignore this command if table already exists.
+     */
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+    /**
+     * @param ifNotExists Quietly ignore this command if table already exists.
+     */
+    public void ifNotExists(boolean ifNotExists) {
+        this.ifNotExists = ifNotExists;
+    }
+
+    /**
+     * @return Extra WITH-params.
+     */
+    public List<String> params() {
+        return params;
+    }
+
+    /**
+     * @param params Extra WITH-params.
+     */
+    public void params(List<String> params) {
+        this.params = params;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java
new file mode 100644
index 0000000..34cb6fe
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.sql;
+
+/**
+ * DROP TABLE statement.
+ */
+public class GridSqlDropTable extends GridSqlStatement {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Quietly ignore this command if table does not exist. */
+    private boolean ifExists;
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Quietly ignore this command if table does not exist.
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @param ifExists Quietly ignore this command if table does not exist.
+     */
+    public void ifExists(boolean ifExists) {
+        this.ifExists = ifExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 71c77ac..f310e0f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import javax.cache.CacheException;
@@ -32,11 +33,18 @@ import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.F;
 import org.h2.command.Command;
 import org.h2.command.CommandContainer;
 import org.h2.command.Prepared;
+import org.h2.command.ddl.AlterTableAddConstraint;
 import org.h2.command.ddl.CreateIndex;
+import org.h2.command.ddl.CreateTable;
+import org.h2.command.ddl.CreateTableData;
+import org.h2.command.ddl.DefineCommand;
 import org.h2.command.ddl.DropIndex;
+import org.h2.command.ddl.DropTable;
 import org.h2.command.ddl.SchemaCommand;
 import org.h2.command.dml.Delete;
 import org.h2.command.dml.Explain;
@@ -46,6 +54,7 @@ import org.h2.command.dml.Query;
 import org.h2.command.dml.Select;
 import org.h2.command.dml.SelectUnion;
 import org.h2.command.dml.Update;
+import org.h2.engine.Constants;
 import org.h2.engine.FunctionAlias;
 import org.h2.expression.Aggregate;
 import org.h2.expression.Alias;
@@ -353,6 +362,47 @@ public class GridSqlQueryParser {
     private static final Getter<SchemaCommand, Schema> SCHEMA_COMMAND_SCHEMA = getter(SchemaCommand.class, "schema");
 
     /** */
+    private static final Getter<CreateTable, CreateTableData> CREATE_TABLE_DATA = getter(CreateTable.class, "data");
+
+    /** */
+    private static final Getter<CreateTable, ArrayList<DefineCommand>> CREATE_TABLE_CONSTRAINTS =
+        getter(CreateTable.class, "constraintCommands");
+
+    /** */
+    private static final Getter<CreateTable, IndexColumn[]> CREATE_TABLE_PK = getter(CreateTable.class,
+        "pkColumns");
+
+    /** */
+    private static final Getter<CreateTable, Boolean> CREATE_TABLE_IF_NOT_EXISTS = getter(CreateTable.class,
+        "ifNotExists");
+
+    /** */
+    private static final Getter<CreateTable, Query> CREATE_TABLE_QUERY = getter(CreateTable.class, "asQuery");
+
+    /** */
+    private static final Getter<DropTable, Boolean> DROP_TABLE_IF_EXISTS = getter(DropTable.class, "ifExists");
+
+    /** */
+    private static final Getter<DropTable, String> DROP_TABLE_NAME = getter(DropTable.class, "tableName");
+
+    /** */
+    private static final Getter<Column, Boolean> COLUMN_IS_COMPUTED = getter(Column.class, "isComputed");
+
+    /** */
+    private static final Getter<Column, Expression> COLUMN_CHECK_CONSTRAINT = getter(Column.class, "checkConstraint");
+
+    /** */
+    private static final String PARAM_NAME_VALUE_SEPARATOR = "=";
+
+    /** */
+    private static final String PARAM_CACHE_TEMPLATE = "cacheTemplate";
+
+    /** Names of the params that need to be present in WITH clause of CREATE TABLE. */
+    private static final String[] MANDATORY_CREATE_TABLE_PARAMS = {
+        PARAM_CACHE_TEMPLATE
+    };
+
+    /** */
     private final IdentityHashMap<Object, Object> h2ObjToGridObj = new IdentityHashMap<>();
 
     /** */
@@ -781,6 +831,210 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * Parse {@code CREATE TABLE} statement.
+     *
+     * @param createTbl {@code CREATE TABLE} statement.
+     * @see <a href="http://h2database.com/html/grammar.html#create_table">H2 {@code CREATE TABLE} spec.</a>
+     */
+    private GridSqlCreateTable parseCreateTable(CreateTable createTbl) {
+        GridSqlCreateTable res = new GridSqlCreateTable();
+
+        Query qry = CREATE_TABLE_QUERY.get(createTbl);
+
+        if (qry != null)
+            throw new IgniteSQLException("CREATE TABLE ... AS ... syntax is not supported",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        List<DefineCommand> constraints = CREATE_TABLE_CONSTRAINTS.get(createTbl);
+
+        if (constraints.size() == 0)
+            throw new IgniteSQLException("No PRIMARY KEY defined for CREATE TABLE",
+                IgniteQueryErrorCode.PARSING);
+
+        if (constraints.size() > 1)
+            throw new IgniteSQLException("Too many constraints - only PRIMARY KEY is supported for CREATE TABLE",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        DefineCommand constraint = constraints.get(0);
+
+        if (!(constraint instanceof AlterTableAddConstraint))
+            throw new IgniteSQLException("Unsupported type of constraint for CREATE TABLE - only PRIMARY KEY " +
+                "is supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        AlterTableAddConstraint alterTbl = (AlterTableAddConstraint)constraint;
+
+        if (alterTbl.getType() != Command.ALTER_TABLE_ADD_CONSTRAINT_PRIMARY_KEY)
+            throw new IgniteSQLException("Unsupported type of constraint for CREATE TABLE - only PRIMARY KEY " +
+                "is supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(createTbl);
+
+        res.schemaName(schema.getName());
+
+        CreateTableData data = CREATE_TABLE_DATA.get(createTbl);
+
+        LinkedHashMap<String, GridSqlColumn> cols = new LinkedHashMap<>(data.columns.size());
+
+        for (Column col : data.columns) {
+            if (col.isAutoIncrement())
+                throw new IgniteSQLException("AUTO_INCREMENT columns are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (!col.isNullable())
+                throw new IgniteSQLException("Non nullable columns are forbidden [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.PARSING);
+
+            if (COLUMN_IS_COMPUTED.get(col))
+                throw new IgniteSQLException("Computed columns are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (col.getDefaultExpression() != null)
+                throw new IgniteSQLException("DEFAULT expressions are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (col.getSequence() != null)
+                throw new IgniteSQLException("SEQUENCE columns are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (col.getSelectivity() != Constants.SELECTIVITY_DEFAULT)
+                throw new IgniteSQLException("SELECTIVITY column attr is not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (COLUMN_CHECK_CONSTRAINT.get(col) != null)
+                throw new IgniteSQLException("Column CHECK constraints are not supported [colName=" + col.getName() +
+                    ']', IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            GridSqlColumn gridCol = new GridSqlColumn(col, null, col.getName());
+
+            gridCol.resultType(GridSqlType.fromColumn(col));
+
+            cols.put(col.getName(), gridCol);
+        }
+
+        if (cols.containsKey(QueryUtils.KEY_FIELD_NAME.toUpperCase()) ||
+            cols.containsKey(QueryUtils.VAL_FIELD_NAME.toUpperCase()))
+            throw new IgniteSQLException("Direct specification of _KEY and _VAL columns is forbidden",
+                IgniteQueryErrorCode.PARSING);
+
+        IndexColumn[] pkIdxCols = CREATE_TABLE_PK.get(createTbl);
+
+        if (F.isEmpty(pkIdxCols))
+            throw new AssertionError("No PRIMARY KEY columns specified");
+
+        LinkedHashSet<String> pkCols = new LinkedHashSet<>();
+
+        for (IndexColumn pkIdxCol : pkIdxCols) {
+            GridSqlColumn gridCol = cols.get(pkIdxCol.columnName);
+
+            assert gridCol != null;
+
+            pkCols.add(gridCol.columnName());
+        }
+
+        int valColsNum = cols.size() - pkCols.size();
+
+        if (valColsNum == 0)
+            throw new IgniteSQLException("No cache value related columns found");
+
+        res.columns(cols);
+
+        res.primaryKeyColumns(pkCols);
+
+        res.tableName(data.tableName);
+
+        res.ifNotExists(CREATE_TABLE_IF_NOT_EXISTS.get(createTbl));
+
+        List<String> extraParams = data.tableEngineParams;
+
+        res.params(extraParams);
+
+        Map<String, String> params = new HashMap<>();
+
+        if (!F.isEmpty(extraParams)) {
+            for (String p : extraParams) {
+                String[] parts = p.split(PARAM_NAME_VALUE_SEPARATOR);
+
+                if (parts.length > 2)
+                    throw new IgniteSQLException("Invalid param syntax: key[=value] expected [paramStr=" + p + ']',
+                        IgniteQueryErrorCode.PARSING);
+
+                String name = parts[0];
+
+                String val = parts.length > 1 ? parts[1] : null;
+
+                if (F.isEmpty(name))
+                    throw new IgniteSQLException("Invalid param syntax: no name given [paramStr=" + p + ']',
+                        IgniteQueryErrorCode.PARSING);
+
+                params.put(name, val);
+            }
+        }
+
+        for (String mandParamName : MANDATORY_CREATE_TABLE_PARAMS) {
+            if (!params.containsKey(mandParamName))
+                throw new IgniteSQLException("Mandatory param is missing [paramName=" + mandParamName + ']');
+        }
+
+        for (Map.Entry<String, String> e : params.entrySet())
+            processExtraParam(e.getKey(), e.getValue(), res);
+
+        return res;
+    }
+
+    /**
+     * Parse {@code DROP TABLE} statement.
+     *
+     * @param dropTbl {@code DROP TABLE} statement.
+     * @see <a href="http://h2database.com/html/grammar.html#drop_table">H2 {@code DROP TABLE} spec.</a>
+     */
+    private GridSqlDropTable parseDropTable(DropTable dropTbl) {
+        GridSqlDropTable res = new GridSqlDropTable();
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(dropTbl);
+
+        res.schemaName(schema.getName());
+
+        res.ifExists(DROP_TABLE_IF_EXISTS.get(dropTbl));
+
+        res.tableName(DROP_TABLE_NAME.get(dropTbl));
+
+        return res;
+    }
+
+    /**
+     * @param name Param name.
+     * @param val Param value.
+     * @param res Table params to update.
+     */
+    private static void processExtraParam(String name, String val, GridSqlCreateTable res) {
+        assert !F.isEmpty(name);
+
+        switch (name) {
+            case PARAM_CACHE_TEMPLATE:
+                ensureParamValueNotEmpty(PARAM_CACHE_TEMPLATE, val);
+
+                res.templateCacheName(val);
+
+                break;
+
+            default:
+                throw new IgniteSQLException("Unknown CREATE TABLE param [paramName=" + name + ']',
+                    IgniteQueryErrorCode.PARSING);
+        }
+    }
+
+    /**
+     * Check that param with mandatory value has it specified.
+     * @param name Param name.
+     * @param val Param value to check.
+     */
+    private static void ensureParamValueNotEmpty(String name, String val) {
+        if (F.isEmpty(val))
+            throw new IgniteSQLException("No value has been given for a CREATE TABLE param [paramName=" + name + ']',
+                IgniteQueryErrorCode.PARSING);
+    }
+
+    /**
      * @param sortOrder Sort order.
      * @param qry Query.
      */
@@ -849,6 +1103,12 @@ public class GridSqlQueryParser {
         if (stmt instanceof DropIndex)
             return parseDropIndex((DropIndex)stmt);
 
+        if (stmt instanceof CreateTable)
+            return parseCreateTable((CreateTable)stmt);
+
+        if (stmt instanceof DropTable)
+            return parseDropTable((DropTable) stmt);
+
         throw new CacheException("Unsupported SQL statement: " + stmt);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
new file mode 100644
index 0000000..b072fdb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -0,0 +1,320 @@
+/*
+ * 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.index;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Tests for CREATE/DROP TABLE.
+ */
+public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
+    /** Client node index. */
+    private final static int CLIENT = 2;
+
+    /** */
+    private final static String INDEXED_CACHE_NAME = CACHE_NAME + "_idx";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        for (IgniteConfiguration cfg : configurations())
+            Ignition.start(cfg);
+
+        client().getOrCreateCache(cacheConfigurationForIndexing());
+
+        client().addCacheConfiguration(cacheConfiguration());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        client().destroyCache("Person");
+
+        super.afterTest();
+    }
+
+    /**
+     * Test that {@code CREATE TABLE} actually creates new cache, H2 table and type descriptor on all nodes.
+     * @throws Exception if failed.
+     */
+    public void testCreateTable() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        for (int i = 0; i < 4; i++) {
+            IgniteEx node = grid(i);
+
+            assertNotNull(node.cache("Person"));
+
+            QueryTypeDescriptorImpl desc = typeExisting(node, "Person", "Person");
+
+            assertEquals(Object.class, desc.keyClass());
+
+            assertEquals("PersonKey", desc.keyTypeName());
+
+            assertEquals(Object.class, desc.valueClass());
+
+            assertEquals("Person", desc.valueTypeName());
+
+            assertEquals(
+                F.asList("id", "city", "name", "surname", "age"),
+                new ArrayList<>(desc.fields().keySet())
+            );
+
+            assertProperty(desc, "id", Integer.class, true);
+
+            assertProperty(desc, "city", String.class, true);
+
+            assertProperty(desc, "name", String.class, false);
+
+            assertProperty(desc, "surname", String.class, false);
+
+            assertProperty(desc, "age", Integer.class, false);
+
+            GridH2Table tbl = ((IgniteH2Indexing)node.context().query().getIndexing()).dataTable("Person", "Person");
+
+            assertNotNull(tbl);
+        }
+    }
+
+    /**
+     * Test that attempting to {@code CREATE TABLE} that already exists does not yield an error if the statement
+     *     contains {@code IF NOT EXISTS} clause.
+     * @throws Exception if failed.
+     */
+    public void testCreateTableIfNotExists() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        cache().query(new SqlFieldsQuery("CREATE TABLE IF NOT EXISTS \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+    }
+
+    /**
+     * Test that attempting to {@code CREATE TABLE} that already exists yields an error.
+     * @throws Exception if failed.
+     */
+    public void testCreateExistingTable() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar" +
+                    ", \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+                    "\"cacheTemplate=cache\""));
+
+                return null;
+            }
+        }, IgniteSQLException.class, "Table already exists: Person");
+    }
+
+    /**
+     * Test that {@code DROP TABLE} actually removes specified cache and type descriptor on all nodes.
+     * @throws Exception if failed.
+     */
+    public void testDropTable() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE IF NOT EXISTS \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        cache().query(new SqlFieldsQuery("DROP TABLE \"Person\".\"Person\""));
+
+        for (int i = 0; i < 4; i++) {
+            IgniteEx node = grid(i);
+
+            assertNull(node.cache("Person"));
+
+            QueryTypeDescriptorImpl desc = type(node, "Person", "Person");
+
+            assertNull(desc);
+        }
+    }
+
+    /**
+     * Test that attempting to {@code DROP TABLE} that does not exist does not yield an error if the statement contains
+     *     {@code IF EXISTS} clause.
+     * @throws Exception if failed.
+     */
+    public void testDropMissingTableIfExists() throws Exception {
+        cache().query(new SqlFieldsQuery("DROP TABLE IF EXISTS \"cache_idx\".\"City\""));
+    }
+
+    /**
+     * Test that attempting to {@code DROP TABLE} that does not exist yields an error.
+     * @throws Exception if failed.
+     */
+    public void testDropMissingTable() throws Exception {
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                cache().query(new SqlFieldsQuery("DROP TABLE \"cache_idx\".\"City\""));
+
+                return null;
+            }
+        }, IgniteSQLException.class, "Table doesn't exist: City");
+    }
+
+    /**
+     * Check that a property in given descriptor is present and has parameters as expected.
+     * @param desc Descriptor.
+     * @param name Property name.
+     * @param type Expected property type.
+     * @param isKey {@code true} if the property is expected to belong to key, {@code false} is it's expected to belong
+     *     to value.
+     */
+    private void assertProperty(QueryTypeDescriptorImpl desc, String name, Class<?> type, boolean isKey) {
+        GridQueryProperty p = desc.property(name);
+
+        assertNotNull(name, p);
+
+        assertEquals(type, p.type());
+
+        assertEquals(isKey, p.key());
+    }
+
+    /**
+     * Get configurations to be used in test.
+     *
+     * @return Configurations.
+     * @throws Exception If failed.
+     */
+    private List<IgniteConfiguration> configurations() throws Exception {
+        return Arrays.asList(
+            serverConfiguration(0),
+            serverConfiguration(1),
+            clientConfiguration(2),
+            serverConfiguration(3)
+        );
+    }
+
+    /**
+     * Create server configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration serverConfiguration(int idx) throws Exception {
+        return commonConfiguration(idx);
+    }
+
+    /**
+     * Create client configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration clientConfiguration(int idx) throws Exception {
+        return commonConfiguration(idx).setClientMode(true);
+    }
+
+    /**
+     * Create common node configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration commonConfiguration(int idx) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(getTestIgniteInstanceName(idx));
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return optimize(cfg);
+    }
+
+    /**
+     * @return Client node.
+     */
+    private IgniteEx client() {
+        return grid(CLIENT);
+    }
+
+    /**
+     * @return Cache to issue queries upon.
+     */
+    private IgniteCache<?, ?> cache() {
+        return client().cache(INDEXED_CACHE_NAME);
+    }
+
+    /**
+     * @return Default cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>(CACHE_NAME);
+
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg.setSqlEscapeAll(true);
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+
+        return ccfg;
+    }
+
+    /**
+     * @return Cache configuration with query entities - unfortunately, we need this to enable indexing at all.
+     */
+    private CacheConfiguration cacheConfigurationForIndexing() {
+        CacheConfiguration<?, ?> ccfg = cacheConfiguration();
+
+        ccfg.setName(INDEXED_CACHE_NAME);
+
+        ccfg.setQueryEntities(Collections.singletonList(
+            new QueryEntity()
+                .setKeyType(Integer.class.getName())
+                .setValueType(Integer.class.getName())
+        ));
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index 9838084..5939b59 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -22,8 +22,11 @@ import java.sql.Connection;
 import java.sql.Date;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
@@ -40,17 +43,23 @@ import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
 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 org.h2.command.Parser;
 import org.h2.command.Prepared;
+import org.h2.command.ddl.CreateTable;
 import org.h2.engine.Session;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.message.DbException;
+import org.h2.table.Column;
+import org.h2.value.Value;
 import org.jetbrains.annotations.NotNull;
 
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
@@ -532,7 +541,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
      *
      */
     public void testParseDropIndex() throws Exception {
-        // Schema that is not set defaults to default schema of connection which is empty string
+        // Schema that is not set defaults to default schema of connection which is sch1
         assertDropIndexEquals(buildDropIndex("idx", "sch1", false), "drop index idx");
         assertDropIndexEquals(buildDropIndex("idx", "sch1", true), "drop index if exists idx");
         assertDropIndexEquals(buildDropIndex("idx", "sch1", true), "drop index if exists sch1.idx");
@@ -546,6 +555,75 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testParseDropTable() throws Exception {
+        // Schema that is not set defaults to default schema of connection which is sch1
+        assertDropTableEquals(buildDropTable("sch1", "tbl", false), "drop table tbl");
+        assertDropTableEquals(buildDropTable("sch1", "tbl", true), "drop table if exists tbl");
+        assertDropTableEquals(buildDropTable("sch1", "tbl", true), "drop table if exists sch1.tbl");
+        assertDropTableEquals(buildDropTable("sch1", "tbl", false), "drop table sch1.tbl");
+
+        // Message is null as long as it may differ from system to system, so we just check for exceptions
+        assertParseThrows("drop table schema2.", DbException.class, null);
+        assertParseThrows("drop table", DbException.class, null);
+        assertParseThrows("drop table if exists", DbException.class, null);
+        assertParseThrows("drop table if exists schema2.", DbException.class, null);
+    }
+
+    /** */
+    public void testParseCreateTable() throws Exception {
+        assertCreateTableEquals(
+            buildCreateTable("sch1", "Person", "cache", F.asList("id", "city"),
+                true, c("id", Value.INT), c("city", Value.STRING), c("name", Value.STRING),
+                c("surname", Value.STRING), c("age", Value.INT)),
+            "CREATE TABLE IF NOT EXISTS sch1.\"Person\" (\"id\" integer, \"city\" varchar," +
+                " \"name\" varchar, \"surname\" varchar, \"age\" integer, PRIMARY KEY (\"id\", \"city\")) WITH " +
+                "\"cacheTemplate=cache\"");
+
+        assertCreateTableEquals(
+            buildCreateTable("sch1", "Person", "cache", F.asList("id"),
+                false, c("id", Value.INT), c("city", Value.STRING), c("name", Value.STRING),
+                c("surname", Value.STRING), c("age", Value.INT)),
+            "CREATE TABLE sch1.\"Person\" (\"id\" integer PRIMARY KEY, \"city\" varchar," +
+                " \"name\" varchar, \"surname\" varchar, \"age\" integer) WITH " +
+                "\"cacheTemplate=cache\"");
+
+        assertParseThrows("create table Person (id int)",
+            IgniteSQLException.class, "No PRIMARY KEY defined for CREATE TABLE");
+
+        assertParseThrows("create table Person (id int) AS SELECT 2 * 2",
+            IgniteSQLException.class, "CREATE TABLE ... AS ... syntax is not supported");
+
+        assertParseThrows("create table Person (id int primary key)",
+            IgniteSQLException.class, "No cache value related columns found");
+
+        assertParseThrows("create table Person (id int primary key, age int null)",
+            IgniteSQLException.class, "Mandatory param is missing [paramName=cacheTemplate]");
+
+        assertParseThrows("create table Person (id int primary key, age int not null) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Non nullable columns are forbidden");
+
+        assertParseThrows("create table Person (id int primary key, age int unique) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Too many constraints - only PRIMARY KEY is supported for CREATE TABLE");
+
+        assertParseThrows("create table Person (id int auto_increment primary key, age int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "AUTO_INCREMENT columns are not supported");
+
+        assertParseThrows("create table Person (id int primary key check id > 0, age int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Column CHECK constraints are not supported [colName=ID]");
+
+        assertParseThrows("create table Person (id int as age * 2 primary key, age int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Computed columns are not supported [colName=ID]");
+
+        assertParseThrows("create table Person (id int primary key, age int default 5) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "DEFAULT expressions are not supported [colName=AGE]");
+
+        assertParseThrows("create table Int (_key int primary key, _val int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Direct specification of _KEY and _VAL columns is forbidden");
+    }
+
+    /**
      * @param sql Statement.
      * @param exCls Exception class.
      * @param msg Expected message.
@@ -610,6 +688,112 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Parse SQL and compare it to expected instance of DROP TABLE.
+     */
+    private void assertCreateTableEquals(GridSqlCreateTable exp, String sql) throws Exception {
+        Prepared prepared = parse(sql);
+
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+
+        assertTrue(stmt instanceof GridSqlCreateTable);
+
+        assertCreateTableEquals(exp, (GridSqlCreateTable) stmt);
+    }
+
+    /**
+     * Test two instances of {@link GridSqlDropTable} for equality.
+     */
+    private static void assertCreateTableEquals(GridSqlCreateTable exp, GridSqlCreateTable actual) {
+        assertEqualsIgnoreCase(exp.schemaName(), actual.schemaName());
+        assertEqualsIgnoreCase(exp.tableName(), actual.tableName());
+        assertEquals(exp.templateCacheName(), actual.templateCacheName());
+        assertEquals(exp.primaryKeyColumns(), actual.primaryKeyColumns());
+        assertEquals(new ArrayList<>(exp.columns().keySet()), new ArrayList<>(actual.columns().keySet()));
+
+        for (Map.Entry<String, GridSqlColumn> col : exp.columns().entrySet()) {
+            GridSqlColumn val = actual.columns().get(col.getKey());
+
+            assertNotNull(val);
+
+            assertEquals(col.getValue().columnName(), val.columnName());
+            assertEquals(col.getValue().column().getType(), val.column().getType());
+        }
+
+        assertEquals(exp.ifNotExists(), actual.ifNotExists());
+    }
+
+    /**
+     *
+     */
+    private static GridSqlCreateTable buildCreateTable(String schema, String tbl, String tplCacheName,
+        Collection<String> pkColNames, boolean ifNotExists, GridSqlColumn... cols) {
+        GridSqlCreateTable res = new GridSqlCreateTable();
+
+        res.schemaName(schema);
+
+        res.tableName(tbl);
+
+        res.templateCacheName(tplCacheName);
+
+        res.primaryKeyColumns(new LinkedHashSet<>(pkColNames));
+
+        LinkedHashMap<String, GridSqlColumn> m = new LinkedHashMap<>();
+
+        for (GridSqlColumn col : cols)
+            m.put(col.columnName(), col);
+
+        res.columns(m);
+
+        res.ifNotExists(ifNotExists);
+
+        return res;
+    }
+
+    /**
+     * @param name Column name.
+     * @param type Column data type.
+     * @return {@link GridSqlColumn} with given name and type.
+     */
+    private static GridSqlColumn c(String name, int type) {
+        return new GridSqlColumn(new Column(name, type), null, name);
+    }
+
+    /**
+     * Parse SQL and compare it to expected instance of DROP TABLE.
+     */
+    private void assertDropTableEquals(GridSqlDropTable exp, String sql) throws Exception {
+        Prepared prepared = parse(sql);
+
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+
+        assertTrue(stmt instanceof GridSqlDropTable);
+
+        assertDropTableEquals(exp, (GridSqlDropTable) stmt);
+    }
+
+    /**
+     * Test two instances of {@link GridSqlDropTable} for equality.
+     */
+    private static void assertDropTableEquals(GridSqlDropTable exp, GridSqlDropTable actual) {
+        assertEqualsIgnoreCase(exp.schemaName(), actual.schemaName());
+        assertEqualsIgnoreCase(exp.tableName(), actual.tableName());
+        assertEquals(exp.ifExists(), actual.ifExists());
+    }
+
+    /**
+     *
+     */
+    private static GridSqlDropTable buildDropTable(String schema, String tbl, boolean ifExists) {
+        GridSqlDropTable res = new GridSqlDropTable();
+
+        res.schemaName(schema);
+        res.tableName(tbl);
+        res.ifExists(ifExists);
+
+        return res;
+    }
+
+    /**
      * Test two instances of {@link GridSqlCreateIndex} for equality.
      */
     private static void assertCreateIndexEquals(GridSqlCreateIndex exp, GridSqlCreateIndex actual) {
@@ -723,6 +907,87 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param createTbl {@code CREATE TABLE} command.
+     * @return Corresponding SQL.
+     */
+    private static String createTableToSql(GridSqlCreateTable createTbl) {
+        GridStringBuilder b = new SB("CREATE TABLE ")
+            .a(createTbl.ifNotExists() ? "IF NOT EXISTS " : "")
+            .a("\n")
+            .a(Parser.quoteIdentifier(createTbl.schemaName()))
+            .a('.')
+            .a(Parser.quoteIdentifier(createTbl.tableName()))
+            .a("\n(");
+
+        boolean singleColPk = false;
+
+        boolean first = true;
+
+        for (GridSqlColumn col : createTbl.columns().values()) {
+            if (!first)
+                b.a(",\n");
+            else
+                first = false;
+
+            if (col.column().isPrimaryKey()) {
+                // Only one column may be marked PRIMARY KEY - multi-col PK is defined separately
+                assert !singleColPk;
+
+                singleColPk = true;
+            }
+
+            b.a('\t')
+                .a(col.getSQL())
+                .a(' ')
+                .a(col.resultType().sql())
+                .a(col.column().isPrimaryKey() ? " PRIMARY KEY" : "");
+        }
+
+        first = true;
+
+        if (!singleColPk && !F.isEmpty(createTbl.primaryKeyColumns())) {
+            b.a(",\n")
+                .a('\t')
+                .a("PRIMARY KEY (\n");
+
+            for (String col : createTbl.primaryKeyColumns()) {
+                GridSqlColumn pkCol = createTbl.columns().get(col);
+
+                assert pkCol != null;
+
+                if (!first)
+                    b.a(",\n");
+                else
+                    first = false;
+
+                b.a("\t\t")
+                    .a(pkCol.getSQL());
+            }
+
+            b.a("\n\t)");
+        }
+
+        b.a("\n)");
+
+        if (!F.isEmpty(createTbl.params())) {
+            b.a("\nWITH ");
+
+            first = true;
+
+            for (String p : createTbl.params()) {
+                if (!first)
+                    b.a(',');
+                else
+                    first = false;
+
+                b.a(Parser.quoteIdentifier(p));
+            }
+        }
+
+        return b.toString();
+    }
+
+    /**
      * @param qry Query.
      */
     private void checkQuery(String qry) throws Exception {
@@ -737,6 +1002,23 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         assertSqlEquals(U.firstNotNull(prepared.getPlanSQL(), prepared.getSQL()), res);
     }
 
+    /**
+     * @param qry Query.
+     */
+    private void checkCreateTable(String qry) throws Exception {
+        Prepared prepared = parse(qry);
+
+        assertTrue(prepared instanceof CreateTable);
+
+        GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(prepared);
+
+        String res = createTableToSql((GridSqlCreateTable)gridStmt);
+
+        System.out.println(normalizeSql(res));
+
+        assertSqlEquals(U.firstNotNull(prepared.getPlanSQL(), prepared.getSQL()), res);
+    }
+
     @QuerySqlFunction
     public static int cool1() {
         return 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 6370cd0..14fb6ce 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -96,6 +96,7 @@ import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoord
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerBasicSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest;
 import org.apache.ignite.internal.processors.cache.index.QueryEntityValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
@@ -227,6 +228,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(H2DynamicIndexAtomicReplicatedSelfTest.class);
         suite.addTestSuite(H2DynamicIndexAtomicPartitionedSelfTest.class);
         suite.addTestSuite(H2DynamicIndexAtomicPartitionedNearSelfTest.class);
+        suite.addTestSuite(H2DynamicTableSelfTest.class);
 
         // Fields queries.
         suite.addTestSuite(SqlFieldsQuerySelfTest.class);