You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/20 08:39:37 UTC

[30/71] [abbrv] ignite git commit: IGNITE-4565: Implemented CREATE INDEX and DROP INDEX. This closes #1773. This closes #1804.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexTransactionalReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexTransactionalReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexTransactionalReplicatedSelfTest.java
new file mode 100644
index 0000000..85a8837
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexTransactionalReplicatedSelfTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/** */
+public class H2DynamicIndexTransactionalReplicatedSelfTest extends H2DynamicIndexAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean nearCache() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java
new file mode 100644
index 0000000..95ad2f1
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java
@@ -0,0 +1,589 @@
+/*
+ * 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 org.apache.ignite.IgniteClientDisconnectedException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.GridTestUtils;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Tests for schema exchange between nodes.
+ */
+public class SchemaExchangeSelfTest extends AbstractSchemaSelfTest {
+    /** Node on which filter should be applied (if any). */
+    private static String filterNodeName;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        filterNodeName = null;
+
+        super.afterTest();
+    }
+
+    /**
+     * Test propagation of empty query schema for static cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testEmptyStatic() throws Exception {
+        checkEmpty(false);
+    }
+
+    /**
+     * Test propagation of empty query schema for dynamic cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testEmptyDynamic() throws Exception {
+        checkEmpty(true);
+    }
+
+    /**
+     * Check empty metadata propagation.
+     *
+     * @param dynamic Dynamic start flag.
+     * @throws Exception If failed.
+     */
+    private void checkEmpty(boolean dynamic) throws Exception {
+        IgniteEx node1;
+
+        if (dynamic) {
+            node1 = startNoCache(1);
+
+            node1.getOrCreateCache(cacheConfiguration());
+        }
+        else
+            node1 = start(1);
+
+        assertTypes(node1);
+
+        IgniteEx node2 = start(2, KeyClass.class, ValueClass.class);
+
+        assertTypes(node1);
+        assertTypes(node2);
+
+        IgniteEx node3 = start(3, KeyClass.class, ValueClass.class, KeyClass2.class, ValueClass2.class);
+
+        assertTypes(node1);
+        assertTypes(node2);
+        assertTypes(node3);
+    }
+
+    /**
+     * Test propagation of non-empty query schema for static cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNonEmptyStatic() throws Exception {
+        checkNonEmpty(false);
+    }
+
+    /**
+     * Test propagation of non-empty query schema for dynamic cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNonEmptyDynamic() throws Exception {
+        checkNonEmpty(true);
+    }
+
+    /**
+     * Check \u0442\u0449\u0442-empty metadata propagation.
+     *
+     * @param dynamic Dynamic start flag.
+     * @throws Exception If failed.
+     */
+    private void checkNonEmpty(boolean dynamic) throws Exception {
+        IgniteEx node1;
+
+        if (dynamic) {
+            node1 = startNoCache(1);
+
+            node1.getOrCreateCache(cacheConfiguration(KeyClass.class, ValueClass.class));
+        }
+        else
+            node1 = start(1, KeyClass.class, ValueClass.class);
+
+        assertTypes(node1, ValueClass.class);
+
+        IgniteEx node2 = start(2);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+
+        IgniteEx node3 = start(3, KeyClass.class, ValueClass.class, KeyClass2.class, ValueClass2.class);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+        assertTypes(node3, ValueClass.class);
+    }
+
+    /**
+     * Make sure that new metadata can be propagated after destroy.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDynamicRestarts() throws Exception {
+        IgniteEx node1 = start(1, KeyClass.class, ValueClass.class);
+        IgniteEx node2 = startNoCache(2);
+        IgniteEx node3 = startClientNoCache(3);
+        IgniteEx node4 = startClientNoCache(4);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+
+        assertTypes(node3);
+
+        node3.cache(CACHE_NAME);
+        assertTypes(node3, ValueClass.class);
+
+        // Check restarts from the first node.
+        node1.destroyCache(CACHE_NAME);
+
+        node1.getOrCreateCache(cacheConfiguration());
+
+        assertTypes(node1);
+        assertTypes(node2);
+        assertTypes(node3);
+
+        node1.destroyCache(CACHE_NAME);
+
+        node1.getOrCreateCache(cacheConfiguration(KeyClass.class, ValueClass.class,
+            KeyClass2.class, ValueClass2.class));
+
+        assertTypes(node1, ValueClass.class, ValueClass2.class);
+        assertTypes(node2, ValueClass.class, ValueClass2.class);
+
+        assertTypes(node3);
+
+        node3.cache(CACHE_NAME);
+        assertTypes(node3, ValueClass.class, ValueClass2.class);
+
+        // Check restarts from the second node.
+        node2.destroyCache(CACHE_NAME);
+
+        node2.getOrCreateCache(cacheConfiguration());
+
+        assertTypes(node1);
+        assertTypes(node2);
+        assertTypes(node3);
+
+        node2.destroyCache(CACHE_NAME);
+
+        node2.getOrCreateCache(cacheConfiguration(KeyClass.class, ValueClass.class,
+            KeyClass2.class, ValueClass2.class));
+
+        assertTypes(node1, ValueClass.class, ValueClass2.class);
+        assertTypes(node2, ValueClass.class, ValueClass2.class);
+
+        assertTypes(node3);
+
+        node3.cache(CACHE_NAME);
+        assertTypes(node3, ValueClass.class, ValueClass2.class);
+
+        assertTypes(node4);
+
+        node4.cache(CACHE_NAME);
+        assertTypes(node4, ValueClass.class, ValueClass2.class);
+
+        // Make sure that joining node observes correct state.
+        assertTypes(start(5), ValueClass.class, ValueClass2.class);
+        assertTypes(startNoCache(6), ValueClass.class, ValueClass2.class);
+
+        assertTypes(startClient(7), ValueClass.class, ValueClass2.class);
+
+        IgniteEx node8 = startClientNoCache(8);
+
+        assertTypes(node8);
+
+        node8.cache(CACHE_NAME);
+        assertTypes(node8, ValueClass.class, ValueClass2.class);
+    }
+
+    /**
+     * Test client join for static cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientJoinStatic() throws Exception {
+        checkClientJoin(false);
+    }
+
+    /**
+     * Test client join for dynamic cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientJoinDynamic() throws Exception {
+        checkClientJoin(true);
+    }
+
+    /**
+     * Check client join.
+     *
+     * @throws Exception If failed.
+     */
+    private void checkClientJoin(boolean dynamic) throws Exception {
+        IgniteEx node1;
+
+        if (dynamic) {
+            node1 = startNoCache(1);
+
+            node1.getOrCreateCache(cacheConfiguration(KeyClass.class, ValueClass.class));
+        }
+        else
+            node1 = start(1, KeyClass.class, ValueClass.class);
+
+        IgniteEx node2 = startClient(2);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+
+        IgniteEx node3 = startClient(3, KeyClass.class, ValueClass.class,
+            KeyClass2.class, ValueClass2.class);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+        assertTypes(node3, ValueClass.class);
+
+        IgniteEx node4 = startClientNoCache(4);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+        assertTypes(node3, ValueClass.class);
+
+        assertTypes(node4);
+
+        node4.cache(CACHE_NAME);
+        assertTypes(node4, ValueClass.class);
+    }
+
+    /**
+     * Test client cache start (static).
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientCacheStartStatic() throws Exception {
+        checkClientCacheStart(false);
+    }
+
+    /**
+     * Test client cache start (dynamic).
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientCacheStartDynamic() throws Exception {
+        checkClientCacheStart(true);
+    }
+
+    /**
+     * Check client cache start.
+     *
+     * @throws Exception If failed.
+     */
+    private void checkClientCacheStart(boolean dynamic) throws Exception {
+        IgniteEx node1 = startNoCache(1);
+
+        IgniteEx node2;
+
+        if (dynamic) {
+            node2 = startClientNoCache(2);
+
+            node2.getOrCreateCache(cacheConfiguration(KeyClass.class, ValueClass.class));
+        }
+        else
+            node2 = startClient(2, KeyClass.class, ValueClass.class);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+
+        IgniteEx node3 = start(3);
+        IgniteEx node4 = start(4,  KeyClass.class, ValueClass.class, KeyClass2.class, ValueClass2.class);
+        IgniteEx node5 = startNoCache(5);
+
+        IgniteEx node6 = startClient(6);
+        IgniteEx node7 = startClient(7,  KeyClass.class, ValueClass.class, KeyClass2.class, ValueClass2.class);
+        IgniteEx node8 = startClientNoCache(8);
+
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+        assertTypes(node3, ValueClass.class);
+        assertTypes(node4, ValueClass.class);
+        assertTypes(node5, ValueClass.class);
+        assertTypes(node6, ValueClass.class);
+        assertTypes(node7, ValueClass.class);
+
+        assertTypes(node8);
+        node8.cache(CACHE_NAME);
+        assertTypes(node8, ValueClass.class);
+
+        node2.destroyCache(CACHE_NAME);
+        node2.getOrCreateCache(
+            cacheConfiguration(KeyClass.class, ValueClass.class, KeyClass2.class, ValueClass2.class));
+
+        assertTypes(node1, ValueClass.class, ValueClass2.class);
+        assertTypes(node2, ValueClass.class, ValueClass2.class);
+        assertTypes(node3, ValueClass.class, ValueClass2.class);
+        assertTypes(node4, ValueClass.class, ValueClass2.class);
+        assertTypes(node5, ValueClass.class, ValueClass2.class);
+
+        assertTypes(node6);
+        assertTypes(node7);
+        assertTypes(node8);
+
+        node6.cache(CACHE_NAME);
+        node7.cache(CACHE_NAME);
+        node8.cache(CACHE_NAME);
+
+        assertTypes(node6, ValueClass.class, ValueClass2.class);
+        assertTypes(node7, ValueClass.class, ValueClass2.class);
+        assertTypes(node8, ValueClass.class, ValueClass2.class);
+    }
+
+    /**
+     * Test behavior when node filter is set.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNodeFilter() throws Exception {
+        filterNodeName = getTestIgniteInstanceName(1);
+
+        IgniteEx node1 = start(1, KeyClass.class, ValueClass.class);
+        assertTypes(node1, ValueClass.class);
+
+        IgniteEx node2 = start(2, KeyClass.class, ValueClass.class);
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+
+        IgniteEx node3 = startNoCache(3);
+        assertTypes(node1, ValueClass.class);
+        assertTypes(node2, ValueClass.class);
+
+        assertTypes(node3);
+
+        node3.cache(CACHE_NAME);
+        assertTypes(node3, ValueClass.class);
+    }
+
+    /**
+     * Test client reconnect.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientReconnect() throws Exception {
+        IgniteEx node1 = start(1, KeyClass.class, ValueClass.class);
+        assertTypes(node1, ValueClass.class);
+
+        IgniteEx node2 = startClientNoCache(2);
+        assertTypes(node2);
+
+        node2.cache(CACHE_NAME);
+        assertTypes(node2, ValueClass.class);
+
+        stopGrid(1);
+
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return grid(2).context().clientDisconnected();
+            }
+        }, 10_000L);
+
+        IgniteFuture reconnFut = null;
+
+        try {   
+            node2.cache(CACHE_NAME);
+
+            fail();
+        }
+        catch (IgniteClientDisconnectedException e) {
+            reconnFut = e.reconnectFuture();
+        }
+
+        node1 = start(1, KeyClass.class, ValueClass.class, KeyClass2.class, ValueClass2.class);
+        assertTypes(node1, ValueClass.class, ValueClass2.class);
+
+        reconnFut.get();
+
+        assertTypes(node2);
+
+        node2.cache(CACHE_NAME);
+        assertTypes(node2, ValueClass.class, ValueClass2.class);
+    }
+
+    /**
+     * Ensure that only provided types exists for the given cache.
+     *
+     * @param node Node.
+     * @param clss Classes.
+     */
+    private static void assertTypes(IgniteEx node, Class... clss) {
+        Map<String, QueryTypeDescriptorImpl> types = types(node, CACHE_NAME);
+
+        if (clss == null || clss.length == 0)
+            assert types.isEmpty();
+        else {
+            assertEquals(clss.length, types.size());
+
+            for (Class cls : clss) {
+                String tblName = tableName(cls);
+
+                assert types.containsKey(tblName);
+            }
+        }
+    }
+
+    /**
+     * Start node with the given cache configuration.
+     *
+     * @param clss Key-value classes.
+     * @return Node.
+     */
+    private IgniteEx start(int idx, Class... clss) throws Exception {
+        return start(idx, false, clss);
+    }
+
+    /**
+     * Start client node with the given cache configuration.
+     *
+     * @param clss Key-value classes.
+     * @return Node.
+     */
+    private IgniteEx startClient(int idx, Class... clss) throws Exception {
+        return start(idx, true, clss);
+    }
+
+    /**
+     * Start node with the given cache configuration.
+     *
+     * @param idx Index.
+     * @param client Client flag.
+     * @param clss Key-value classes.
+     * @return Node.
+     */
+    private IgniteEx start(int idx, boolean client, Class... clss) throws Exception {
+        String name = getTestIgniteInstanceName(idx);
+
+        IgniteConfiguration cfg = getConfiguration(name);
+
+        cfg.setClientMode(client);
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setCacheConfiguration(cacheConfiguration(clss));
+
+        if (filterNodeName != null && F.eq(name, filterNodeName))
+            cfg.setUserAttributes(Collections.singletonMap("AFF_NODE", true));
+
+        return (IgniteEx)Ignition.start(cfg);
+    }
+
+    /**
+     * Start node without cache.
+     *
+     * @param idx Index.
+     * @return Node.
+     * @throws Exception If failed.
+     */
+    private IgniteEx startNoCache(int idx) throws Exception {
+        return startNoCache(idx, false);
+    }
+
+    /**
+     * Start client node without cache.
+     *
+     * @param idx Index.
+     * @return Node.
+     * @throws Exception If failed.
+     */
+    private IgniteEx startClientNoCache(int idx) throws Exception {
+        return startNoCache(idx, true);
+    }
+
+    /**
+     * Start node without cache.
+     *
+     * @param idx Index.
+     * @param client Client mode flag.
+     * @return Node.
+     * @throws Exception If failed.
+     */
+    private IgniteEx startNoCache(int idx, boolean client) throws Exception {
+        String name = getTestIgniteInstanceName(idx);
+
+        IgniteConfiguration cfg = getConfiguration(name);
+
+        cfg.setClientMode(client);
+        cfg.setLocalHost("127.0.0.1");
+
+        return (IgniteEx)Ignition.start(cfg);
+    }
+
+    /**
+     * Get cache configuration.
+     *
+     * @param clss QUery classes.
+     * @return Configuration.
+     */
+    @SuppressWarnings("unchecked")
+    private static CacheConfiguration cacheConfiguration(Class... clss) {
+        CacheConfiguration ccfg = new CacheConfiguration().setName(CACHE_NAME).setIndexedTypes(clss);
+
+        if (filterNodeName != null) {
+            ccfg.setNodeFilter(new IgnitePredicate<ClusterNode>() {
+                @Override public boolean apply(ClusterNode node) {
+                    return node.attribute("AFF_NODE") != null;
+                }
+            });
+        }
+
+        return ccfg;
+    }
+
+    // TODO: Start/stop many nodes with static configs and dynamic start/stop.
+
+    /**
+     * Key class 2.
+     */
+    @SuppressWarnings("unused")
+    private static class KeyClass2 {
+        @QuerySqlField
+        private String keyField2;
+    }
+
+    /**
+     * Value class 2.
+     */
+    @SuppressWarnings("unused")
+    private static class ValueClass2 {
+        @QuerySqlField
+        private String valField2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java
index 6013b80..e0c6396 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java
@@ -85,6 +85,6 @@ public class IgniteCacheLocalQuerySelfTest extends IgniteCacheAbstractQuerySelfT
         List<List<?>> res = cache.query(new SqlFieldsQuery(
             "explain select _key from String where _val > 'value1'").setLocal(true)).getAll();
 
-        assertTrue("__ explain: \n" + res, ((String)res.get(0).get(0)).contains("_val_idx"));
+        assertTrue("__ explain: \n" + res, ((String)res.get(0).get(0)).toLowerCase().contains("_val_idx"));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 711db2f..fe966f9 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
@@ -83,7 +83,6 @@ public class IgniteQueryDedicatedPoolTest extends GridCommonAbstractTest {
         if ("client".equals(gridName))
             cfg.setClientMode(true);
 
-
         cfg.setIndexingSpi(new TestIndexingSpi());
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index b180eba..2a03796 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -413,7 +413,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
 
             info("Plan: " + plan);
 
-            assertTrue(plan.contains("grpIdx"));
+            assertTrue("_explain: " + plan, plan.toLowerCase().contains("grpidx"));
 
             // Sorted list
             List<GroupIndexTestValue> list = F.asList(

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 1c30412..e62199a 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
@@ -87,7 +87,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         return cfg;
     }
 
-    /**
+    /*
      * Fields initialization.
      */
     static {
@@ -246,22 +246,22 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
     public void testSpi() throws Exception {
         IgniteH2Indexing spi = getIndexing();
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA));
-        assertEquals(-1, spi.size(typeAB.space(), typeAB));
-        assertEquals(-1, spi.size(typeBA.space(), typeBA));
+        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(-1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(-1, spi.size(typeBA.space(), typeBA.name()));
 
         IgniteCache<Integer, BinaryObject> cacheA = ignite0.createCache(cacheACfg());
 
-        assertEquals(0, spi.size(typeAA.space(), typeAA));
-        assertEquals(0, spi.size(typeAB.space(), typeAB));
-        assertEquals(-1, spi.size(typeBA.space(), typeBA));
+        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(0, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(-1, spi.size(typeBA.space(), typeBA.name()));
 
         IgniteCache<Integer, BinaryObject> cacheB = ignite0.createCache(cacheBCfg());
 
         // Initially all is empty.
-        assertEquals(0, spi.size(typeAA.space(), typeAA));
-        assertEquals(0, spi.size(typeAB.space(), typeAB));
-        assertEquals(0, spi.size(typeBA.space(), typeBA));
+        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(0, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.space(), 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());
@@ -288,48 +288,48 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         cacheA.put(1, aa("A", 1, "Vasya", 10).build());
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA));
-        assertEquals(0, spi.size(typeAB.space(), typeAB));
-        assertEquals(0, spi.size(typeBA.space(), typeBA));
+        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(0, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.space(), 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));
-        assertEquals(1, spi.size(typeAB.space(), typeAB));
-        assertEquals(0, spi.size(typeBA.space(), typeBA));
+        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
 
         cacheB.put(1, ba(2, "Petya", 25, true).build());
 
         // No replacement because of different space.
-        assertEquals(0, spi.size(typeAA.space(), typeAA));
-        assertEquals(1, spi.size(typeAB.space(), typeAB));
-        assertEquals(1, spi.size(typeBA.space(), typeBA));
+        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
 
         cacheB.put(1, ba(2, "Kolya", 25, true).build());
 
         // Replacement in the same table.
-        assertEquals(0, spi.size(typeAA.space(), typeAA));
-        assertEquals(1, spi.size(typeAB.space(), typeAB));
-        assertEquals(1, spi.size(typeBA.space(), typeBA));
+        assertEquals(0, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
 
         cacheA.put(2, aa("A", 2, "Valera", 19).build());
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA));
-        assertEquals(1, spi.size(typeAB.space(), typeAB));
-        assertEquals(1, spi.size(typeBA.space(), typeBA));
+        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
 
         cacheA.put(3, aa("A", 3, "Borya", 18).build());
 
-        assertEquals(2, spi.size(typeAA.space(), typeAA));
-        assertEquals(1, spi.size(typeAB.space(), typeAB));
-        assertEquals(1, spi.size(typeBA.space(), typeBA));
+        assertEquals(2, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
 
         cacheA.put(4, ab(4, "Vitalya", 20, "Very Good guy").build());
 
-        assertEquals(2, spi.size(typeAA.space(), typeAA));
-        assertEquals(2, spi.size(typeAB.space(), typeAB));
-        assertEquals(1, spi.size(typeBA.space(), typeBA));
+        assertEquals(2, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
 
         // Query data.
         Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> res =
@@ -376,7 +376,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         // Text queries
         Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> txtRes = spi.queryLocalText(typeAB.space(), "good",
-            typeAB, null);
+            typeAB.name(), null);
 
         assertTrue(txtRes.hasNext());
         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(txtRes.next()));
@@ -410,32 +410,32 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         // Remove
         cacheA.remove(2);
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA));
-        assertEquals(2, spi.size(typeAB.space(), typeAB));
-        assertEquals(1, spi.size(typeBA.space(), typeBA));
+        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(1, spi.size(typeBA.space(), typeBA.name()));
 
         cacheB.remove(1);
 
-        assertEquals(1, spi.size(typeAA.space(), typeAA));
-        assertEquals(2, spi.size(typeAB.space(), typeAB));
-        assertEquals(0, spi.size(typeBA.space(), typeBA));
+        assertEquals(1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
 
         // Unregister.
-        spi.unregisterType(typeAA.space(), typeAA);
+        spi.unregisterType(typeAA.space(), typeAA.name());
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA));
-        assertEquals(2, spi.size(typeAB.space(), typeAB));
-        assertEquals(0, spi.size(typeBA.space(), typeBA));
+        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(2, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
 
-        spi.unregisterType(typeAB.space(), typeAB);
+        spi.unregisterType(typeAB.space(), typeAB.name());
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA));
-        assertEquals(-1, spi.size(typeAB.space(), typeAB));
-        assertEquals(0, spi.size(typeBA.space(), typeBA));
+        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
+        assertEquals(-1, spi.size(typeAB.space(), typeAB.name()));
+        assertEquals(0, spi.size(typeBA.space(), typeBA.name()));
 
-        spi.unregisterType(typeBA.space(), typeBA);
+        spi.unregisterType(typeBA.space(), typeBA.name());
 
-        assertEquals(-1, spi.size(typeAA.space(), typeAA));
+        assertEquals(-1, spi.size(typeAA.space(), typeAA.name()));
     }
 
     /**
@@ -500,6 +500,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public String name() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
         @Override public Collection<String> fields() {
             return fields;
         }
@@ -640,8 +645,12 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         /** */
         @Override public Map<String, GridQueryIndexDescriptor> indexes() {
-            return textIdx == null ? Collections.<String, GridQueryIndexDescriptor>emptyMap() :
-                Collections.singletonMap("index", textIdx);
+            return Collections.emptyMap();
+        }
+
+        /** */
+        @Override public GridQueryIndexDescriptor textIndex() {
+            return textIdx;
         }
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/IgniteSqlQueryMinMaxTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/IgniteSqlQueryMinMaxTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/IgniteSqlQueryMinMaxTest.java
index 0f50d7e..e78b695 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/IgniteSqlQueryMinMaxTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/IgniteSqlQueryMinMaxTest.java
@@ -142,8 +142,8 @@ public class IgniteSqlQueryMinMaxTest extends GridCommonAbstractTest {
             QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("explain select min(_key), max(_key) from ValueObj"));
             List<List<?>> result = cursor.getAll();
             assertEquals(2, result.size());
-            assertTrue(((String) result.get(0).get(0)).contains("_key_PK"));
-            assertTrue(((String) result.get(0).get(0)).contains("direct lookup"));
+            assertTrue(((String) result.get(0).get(0)).toLowerCase().contains("_key_pk"));
+            assertTrue(((String) result.get(0).get(0)).toLowerCase().contains("direct lookup"));
         }
     }
 
@@ -158,8 +158,8 @@ public class IgniteSqlQueryMinMaxTest extends GridCommonAbstractTest {
             QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("explain select min(idxVal), max(idxVal) from ValueObj"));
             List<List<?>> result = cursor.getAll();
             assertEquals(2, result.size());
-            assertTrue(((String)result.get(0).get(0)).contains("idxVal_idx"));
-            assertTrue(((String)result.get(0).get(0)).contains("direct lookup"));
+            assertTrue(((String)result.get(0).get(0)).toLowerCase().contains("idxval_idx"));
+            assertTrue(((String)result.get(0).get(0)).toLowerCase().contains("direct lookup"));
         }
     }
 
@@ -174,8 +174,8 @@ public class IgniteSqlQueryMinMaxTest extends GridCommonAbstractTest {
             QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("explain select min(_key), max(_key) from Integer"));
             List<List<?>> result = cursor.getAll();
             assertEquals(2, result.size());
-            assertTrue(((String)result.get(0).get(0)).contains("_key_PK"));
-            assertTrue(((String)result.get(0).get(0)).contains("direct lookup"));
+            assertTrue(((String)result.get(0).get(0)).toLowerCase().contains("_key_pk"));
+            assertTrue(((String)result.get(0).get(0)).toLowerCase().contains("direct lookup"));
         }
     }
 
@@ -190,8 +190,8 @@ public class IgniteSqlQueryMinMaxTest extends GridCommonAbstractTest {
             QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("explain select min(_val), max(_val) from Integer"));
             List<List<?>> result = cursor.getAll();
             assertEquals(2, result.size());
-            assertTrue(((String)result.get(0).get(0)).contains("_val_idx"));
-            assertTrue(((String)result.get(0).get(0)).contains("direct lookup"));
+            assertTrue(((String)result.get(0).get(0)).toLowerCase().contains("_val_idx"));
+            assertTrue(((String)result.get(0).get(0)).toLowerCase().contains("direct lookup"));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
index 4bd9302..88ff61e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
@@ -79,7 +79,7 @@ public class GridH2TableSelfTest extends GridCommonAbstractTest {
     private static final String NON_UNIQUE_IDX_NAME = "__GG_IDX_";
 
     /** */
-    private static final String SCAN_IDX_NAME = GridH2Table.ScanIndex.SCAN_INDEX_NAME_SUFFIX;
+    private static final String SCAN_IDX_NAME = GridH2PrimaryScanIndex.SCAN_INDEX_NAME_SUFFIX;
 
     /** */
     private Connection conn;
@@ -89,31 +89,36 @@ public class GridH2TableSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        Driver.load();
-
-        conn = DriverManager.getConnection(DB_URL);
-
-        tbl = GridH2Table.Engine.createTable(conn, CREATE_TABLE_SQL, null, new GridH2Table.IndexesFactory() {
-            @Override public H2RowFactory createRowFactory(GridH2Table tbl) {
-                return null;
-            }
-
-            @Override public ArrayList<Index> createIndexes(GridH2Table tbl) {
-                ArrayList<Index> idxs = new ArrayList<>();
-
-                IndexColumn id = tbl.indexColumn(0, SortOrder.ASCENDING);
-                IndexColumn t = tbl.indexColumn(1, SortOrder.ASCENDING);
-                IndexColumn str = tbl.indexColumn(2, SortOrder.DESCENDING);
-                IndexColumn x = tbl.indexColumn(3, SortOrder.DESCENDING);
-
-                idxs.add(new H2PkHashIndex(null, tbl, HASH, F.asList(id)));
-                idxs.add(new GridH2TreeIndex(PK_NAME, tbl, true, F.asList(id)));
-                idxs.add(new GridH2TreeIndex(NON_UNIQUE_IDX_NAME, tbl, false, F.asList(x, t, id)));
-                idxs.add(new GridH2TreeIndex(STR_IDX_NAME, tbl, false, F.asList(str, id)));
-
-                return idxs;
-            }
-        }, null);
+        // TODO: IGNITE-4994: Restore mock.
+//        Driver.load();
+//
+//        conn = DriverManager.getConnection(DB_URL);
+//
+//        tbl = GridH2Table.Engine.createTable(conn, CREATE_TABLE_SQL, null, new GridH2Table.IndexesFactory() {
+//            @Override public void onTableCreated(GridH2Table tbl) {
+//                // No-op.
+//            }
+//
+//            @Override public H2RowFactory createRowFactory(GridH2Table tbl) {
+//                return null;
+//            }
+//
+//            @Override public ArrayList<Index> createIndexes(GridH2Table tbl) {
+//                ArrayList<Index> idxs = new ArrayList<>();
+//
+//                IndexColumn id = tbl.indexColumn(0, SortOrder.ASCENDING);
+//                IndexColumn t = tbl.indexColumn(1, SortOrder.ASCENDING);
+//                IndexColumn str = tbl.indexColumn(2, SortOrder.DESCENDING);
+//                IndexColumn x = tbl.indexColumn(3, SortOrder.DESCENDING);
+//
+//                idxs.add(new H2PkHashIndex(null, tbl, HASH, F.asList(id)));
+//                idxs.add(new GridH2TreeIndex(PK_NAME, tbl, true, F.asList(id)));
+//                idxs.add(new GridH2TreeIndex(NON_UNIQUE_IDX_NAME, tbl, false, F.asList(x, t, id)));
+//                idxs.add(new GridH2TreeIndex(STR_IDX_NAME, tbl, false, F.asList(str, id)));
+//
+//                return idxs;
+//            }
+//        }, null);
     }
 
     /** {@inheritDoc} */
@@ -285,120 +290,6 @@ public class GridH2TableSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Multithreaded indexes consistency test.
-     *
-     * @throws Exception If failed.
-     */
-    public void testIndexesMultiThreadedConsistency() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-3484");
-
-        final int threads = 19;
-        final int iterations = 1500;
-
-        multithreaded(new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                Random rnd = new Random();
-
-                PreparedStatement ps1 = null;
-
-                for (int i = 0; i < iterations; i++) {
-                    UUID id = UUID.randomUUID();
-
-                    int x = rnd.nextInt(50);
-
-                    long t = System.currentTimeMillis();
-
-                    GridH2Row row = row(id, t, rnd.nextBoolean() ? id.toString() : UUID.randomUUID().toString(), x);
-
-                    assertTrue(tbl.doUpdate(row, false));
-
-                    if (rnd.nextInt(100) == 0) {
-                        tbl.lock(null, false, false);
-
-                        long cnt = 0;
-
-                        try {
-                            ArrayList<Index> idxs = tbl.getIndexes();
-
-                            // Consistency check.
-                            Set<Row> rowSet = checkIndexesConsistent(idxs, null);
-
-                            // Order check.
-                            checkOrdered(idxs);
-
-                            checkIndexesConsistent(idxs, rowSet);
-
-                            cnt = idxs.get(0).getRowCount(null);
-                        }
-                        finally {
-                            tbl.unlock(null);
-                        }
-
-                        // Row count is valid.
-                        ResultSet rs = conn.createStatement().executeQuery("select count(*) from t");
-
-                        assertTrue(rs.next());
-
-                        int cnt2 = rs.getInt(1);
-
-                        rs.close();
-
-                        assertTrue(cnt2 + " must be >= " + cnt, cnt2 >= cnt);
-                        assertTrue(cnt2 <= threads * iterations);
-
-                        // Search by ID.
-                        rs = conn.createStatement().executeQuery("select * from t where id = '" + id.toString() + "'");
-
-                        assertTrue(rs.next());
-                        assertFalse(rs.next());
-
-                        rs.close();
-
-                        // Scan search.
-                        if (ps1 == null)
-                            ps1 = conn.prepareStatement("select id from t where x = ? order by t desc");
-
-                        ps1.setInt(1, x);
-
-                        rs = ps1.executeQuery();
-
-                        for (;;) {
-                            assertTrue(rs.next());
-
-                            if (rs.getObject(1).equals(id))
-                                break;
-                        }
-
-                        rs.close();
-                    }
-                }
-                return null;
-            }
-        }, threads);
-    }
-
-    /**
-     * Run test in endless loop.
-     *
-     * @param args Arguments.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("InfiniteLoopStatement")
-    public static void main(String ... args) throws Exception {
-        for (int i = 0;;) {
-            GridH2TableSelfTest t = new GridH2TableSelfTest();
-
-            t.beforeTest();
-
-            t.testDataLoss();
-
-            t.afterTest();
-
-            System.out.println("..." + ++i);
-        }
-    }
-
-    /**
       * @throws Exception If failed.
      */
     public void testRangeQuery() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 1d8893e..631adeb 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,9 +22,15 @@ import java.sql.Connection;
 import java.sql.Date;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -32,15 +38,19 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 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.typedef.F;
 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.Prepared;
 import org.h2.engine.Session;
 import org.h2.jdbc.JdbcConnection;
+import org.h2.message.DbException;
 
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -453,6 +463,206 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     /**
      *
      */
+    public void testParseCreateIndex() throws Exception {
+        assertCreateIndexEquals(
+            buildCreateIndex(null, "Person", "sch1", false, QueryIndexType.SORTED, "name", true),
+            "create index on Person (name)");
+
+        assertCreateIndexEquals(
+            buildCreateIndex("idx", "Person", "sch1", false, QueryIndexType.SORTED, "name", true),
+            "create index idx on Person (name ASC)");
+
+        assertCreateIndexEquals(
+            buildCreateIndex("idx", "Person", "sch1", false, QueryIndexType.GEOSPATIAL, "name", true),
+            "create spatial index sch1.idx on sch1.Person (name ASC)");
+
+        assertCreateIndexEquals(
+            buildCreateIndex("idx", "Person", "sch1", true, QueryIndexType.SORTED, "name", true),
+            "create index if not exists sch1.idx on sch1.Person (name)");
+
+        // When we specify schema for the table and don't specify it for the index, resulting schema is table's
+        assertCreateIndexEquals(
+            buildCreateIndex("idx", "Person", "sch1", true, QueryIndexType.SORTED, "name", false),
+            "create index if not exists idx on sch1.Person (name dEsC)");
+
+        assertCreateIndexEquals(
+            buildCreateIndex("idx", "Person", "sch1", true, QueryIndexType.GEOSPATIAL, "old", true, "name", false),
+            "create spatial index if not exists idx on Person (old, name desc)");
+
+        // Schemas for index and table must match
+        assertParseThrows("create index if not exists sch2.idx on sch1.Person (name)",
+            DbException.class, "Schema name must match [90080-191]");
+
+        assertParseThrows("create hash index if not exists idx on Person (name)",
+            IgniteSQLException.class, "Only SPATIAL modifier is supported for CREATE INDEX");
+
+        assertParseThrows("create unique index if not exists idx on Person (name)",
+            IgniteSQLException.class, "Only SPATIAL modifier is supported for CREATE INDEX");
+
+        assertParseThrows("create primary key on Person (name)",
+            IgniteSQLException.class, "Only SPATIAL modifier is supported for CREATE INDEX");
+
+        assertParseThrows("create primary key hash on Person (name)",
+            IgniteSQLException.class, "Only SPATIAL modifier is supported for CREATE INDEX");
+
+        assertParseThrows("create index on Person (name nulls first)",
+            IgniteSQLException.class, "NULLS FIRST and NULLS LAST modifiers are not supported for index columns");
+
+        assertParseThrows("create index on Person (name desc nulls last)",
+            IgniteSQLException.class, "NULLS FIRST and NULLS LAST modifiers are not supported for index columns");
+    }
+
+    /**
+     *
+     */
+    public void testParseDropIndex() throws Exception {
+        // Schema that is not set defaults to default schema of connection which is empty string
+        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");
+        assertDropIndexEquals(buildDropIndex("idx", "sch1", false), "drop index sch1.idx");
+
+        // Message is null as long as it may differ from system to system, so we just check for exceptions
+        assertParseThrows("drop index schema2.", DbException.class, null);
+        assertParseThrows("drop index", DbException.class, null);
+        assertParseThrows("drop index if exists", DbException.class, null);
+        assertParseThrows("drop index if exists schema2.", DbException.class, null);
+    }
+
+    /**
+     * @param sql Statement.
+     * @param exCls Exception class.
+     * @param msg Expected message.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    private void assertParseThrows(final String sql, Class<? extends Exception> exCls, String msg) {
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Prepared p = parse(sql);
+
+                return new GridSqlQueryParser(false).parse(p);
+            }
+        }, exCls, msg);
+    }
+
+    /**
+     * Parse SQL and compare it to expected instance.
+     */
+    private void assertCreateIndexEquals(GridSqlCreateIndex exp, String sql) throws Exception {
+        Prepared prepared = parse(sql);
+
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+
+        assertTrue(stmt instanceof GridSqlCreateIndex);
+
+        assertCreateIndexEquals(exp, (GridSqlCreateIndex) stmt);
+    }
+
+    /**
+     * Parse SQL and compare it to expected instance of DROP INDEX.
+     */
+    private void assertDropIndexEquals(GridSqlDropIndex exp, String sql) throws Exception {
+        Prepared prepared = parse(sql);
+
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+
+        assertTrue(stmt instanceof GridSqlDropIndex);
+
+        assertDropIndexEquals(exp, (GridSqlDropIndex) stmt);
+    }
+
+    /**
+     * Test two instances of {@link GridSqlDropIndex} for equality.
+     */
+    private static void assertDropIndexEquals(GridSqlDropIndex exp, GridSqlDropIndex actual) {
+        assertEqualsIgnoreCase(exp.name(), actual.name());
+        assertEqualsIgnoreCase(exp.schemaName(), actual.schemaName());
+        assertEquals(exp.ifExists(), actual.ifExists());
+    }
+
+    /**
+     *
+     */
+    private static GridSqlDropIndex buildDropIndex(String name, String schema, boolean ifExists) {
+        GridSqlDropIndex res = new GridSqlDropIndex();
+
+        res.name(name);
+        res.schemaName(schema);
+        res.ifExists(ifExists);
+
+        return res;
+    }
+
+    /**
+     * Test two instances of {@link GridSqlCreateIndex} for equality.
+     */
+    private static void assertCreateIndexEquals(GridSqlCreateIndex exp, GridSqlCreateIndex actual) {
+        assertEquals(exp.ifNotExists(), actual.ifNotExists());
+        assertEqualsIgnoreCase(exp.schemaName(), actual.schemaName());
+        assertEqualsIgnoreCase(exp.tableName(), actual.tableName());
+
+        assertEqualsIgnoreCase(exp.index().getName(), actual.index().getName());
+
+        Iterator<Map.Entry<String, Boolean>> expFldsIt = exp.index().getFields().entrySet().iterator();
+        Iterator<Map.Entry<String, Boolean>> actualFldsIt = actual.index().getFields().entrySet().iterator();
+
+        while (expFldsIt.hasNext()) {
+            assertTrue(actualFldsIt.hasNext());
+
+            Map.Entry<String, Boolean> expEntry = expFldsIt.next();
+            Map.Entry<String, Boolean> actualEntry = actualFldsIt.next();
+
+            assertEqualsIgnoreCase(expEntry.getKey(), actualEntry.getKey());
+            assertEquals(expEntry.getValue(), actualEntry.getValue());
+        }
+
+        assertFalse(actualFldsIt.hasNext());
+
+        assertEquals(exp.index().getIndexType(), actual.index().getIndexType());
+    }
+
+    /**
+     *
+     */
+    private static void assertEqualsIgnoreCase(String exp, String actual) {
+        assertEquals((exp == null), (actual == null));
+
+        if (exp != null)
+            assertTrue(exp.equalsIgnoreCase(actual));
+    }
+
+    /**
+     *
+     */
+    private static GridSqlCreateIndex buildCreateIndex(String name, String tblName, String schemaName, boolean ifNotExists,
+        QueryIndexType type, Object... flds) {
+        QueryIndex idx = new QueryIndex();
+
+        idx.setName(name);
+
+        assert !F.isEmpty(flds) && flds.length % 2 == 0;
+
+        LinkedHashMap<String, Boolean> trueFlds = new LinkedHashMap<>();
+
+        for (int i = 0; i < flds.length / 2; i++)
+            trueFlds.put((String)flds[i * 2], (Boolean)flds[i * 2 + 1]);
+
+        idx.setFields(trueFlds);
+        idx.setIndexType(type);
+
+        GridSqlCreateIndex res = new GridSqlCreateIndex();
+
+        res.schemaName(schemaName);
+        res.tableName(tblName);
+        res.ifNotExists(ifNotExists);
+        res.index(idx);
+
+        return res;
+    }
+
+    /**
+     *
+     */
     private JdbcConnection connection() throws Exception {
         GridKernalContext ctx = ((IgniteEx)ignite).context();
 
@@ -508,7 +718,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
 
         System.out.println(normalizeSql(res));
 
-        assertSqlEquals(prepared.getPlanSQL(), res);
+        assertSqlEquals(U.firstNotNull(prepared.getPlanSQL(), prepared.getSQL()), res);
     }
 
     @QuerySqlFunction

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 de341d0..b61affe 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
@@ -81,6 +81,18 @@ import org.apache.ignite.internal.processors.cache.distributed.replicated.Ignite
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexAtomicPartitionedNearSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexAtomicPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexAtomicReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexTransactionalPartitionedNearSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexTransactionalPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexTransactionalReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.index.DynamicIndexClientBasicSelfTest;
+import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoordinatorBasicSelfTest;
+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.SchemaExchangeSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest;
@@ -98,7 +110,6 @@ import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2TableSelfTest;
 import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
@@ -117,8 +128,20 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite");
 
+        // Dynamic index create/drop tests.
+        suite.addTest(new TestSuite(SchemaExchangeSelfTest.class));
+
+        suite.addTest(new TestSuite(DynamicIndexServerCoordinatorBasicSelfTest.class));
+        suite.addTest(new TestSuite(DynamicIndexServerBasicSelfTest.class));
+        suite.addTest(new TestSuite(DynamicIndexServerNodeFilterCoordinatorBasicSelfTest.class));
+        suite.addTest(new TestSuite(DynamicIndexServerNodeFIlterBasicSelfTest.class));
+        suite.addTest(new TestSuite(DynamicIndexClientBasicSelfTest.class));
+
         // H2 tests.
-        suite.addTest(new TestSuite(GridH2TableSelfTest.class));
+
+        // TODO: IGNITE-4994: Restore mock.
+        // suite.addTest(new TestSuite(GridH2TableSelfTest.class));
+
         suite.addTest(new TestSuite(GridH2IndexingInMemSelfTest.class));
         suite.addTest(new TestSuite(GridH2IndexingOffheapSelfTest.class));
 
@@ -171,6 +194,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteCacheMultipleIndexedTypesTest.class);
 
+        // DML.
         suite.addTestSuite(IgniteCacheMergeSqlQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheInsertSqlQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheUpdateSqlQuerySelfTest.class);
@@ -185,6 +209,14 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheMultipleIndexedTypesTest.class);
         suite.addTestSuite(IgniteSqlQueryMinMaxTest.class);
 
+        // DDL.
+        suite.addTestSuite(H2DynamicIndexTransactionalReplicatedSelfTest.class);
+        suite.addTestSuite(H2DynamicIndexTransactionalPartitionedSelfTest.class);
+        suite.addTestSuite(H2DynamicIndexTransactionalPartitionedNearSelfTest.class);
+        suite.addTestSuite(H2DynamicIndexAtomicReplicatedSelfTest.class);
+        suite.addTestSuite(H2DynamicIndexAtomicPartitionedSelfTest.class);
+        suite.addTestSuite(H2DynamicIndexAtomicPartitionedNearSelfTest.class);
+
         // Fields queries.
         suite.addTestSuite(SqlFieldsQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
@@ -227,7 +259,6 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
 
         // Cancellation.
-
         suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class);
         suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 416f150..5085cd5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -30,6 +30,10 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheD
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
+import org.apache.ignite.internal.processors.cache.index.DynamicIndexPartitionedAtomicConcurrentSelfTest;
+import org.apache.ignite.internal.processors.cache.index.DynamicIndexPartitionedTransactionalConcurrentSelfTest;
+import org.apache.ignite.internal.processors.cache.index.DynamicIndexReplicatedAtomicConcurrentSelfTest;
+import org.apache.ignite.internal.processors.cache.index.DynamicIndexReplicatedTransactionalConcurrentSelfTest;
 import org.apache.ignite.testframework.IgniteTestSuite;
 
 /**
@@ -43,6 +47,13 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite 2");
 
+        // Dynamic index create/drop tests.
+        suite.addTestSuite(DynamicIndexPartitionedAtomicConcurrentSelfTest.class);
+        suite.addTestSuite(DynamicIndexPartitionedTransactionalConcurrentSelfTest.class);
+        suite.addTestSuite(DynamicIndexReplicatedAtomicConcurrentSelfTest.class);
+        suite.addTestSuite(DynamicIndexReplicatedTransactionalConcurrentSelfTest.class);
+
+        // Other tests.
         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
 
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);