You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2014/12/09 17:33:12 UTC

[02/13] incubator-ignite git commit: ignite-qry - merged

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
index b2355ee..430c4cb 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
@@ -14,7 +14,6 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
-import org.gridgain.grid.spi.indexing.h2.*;
 import org.gridgain.grid.util.typedef.*;
 
 import java.util.*;
@@ -48,12 +47,6 @@ public class GridCacheReduceQueryMultithreadedSelfTest extends GridCacheAbstract
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 
-        GridH2IndexingSpi indexing = new GridH2IndexingSpi();
-
-        indexing.setDefaultIndexPrimitiveKey(true);
-
-        c.setIndexingSpi(indexing);
-
         c.setMarshaller(new IgniteOptimizedMarshaller(false));
 
         return c;
@@ -67,6 +60,12 @@ public class GridCacheReduceQueryMultithreadedSelfTest extends GridCacheAbstract
         cfg.setBackups(1);
         cfg.setWriteSynchronizationMode(FULL_SYNC);
 
+        GridCacheQueryConfiguration qcfg = new GridCacheQueryConfiguration();
+
+        qcfg.setIndexPrimitiveKey(true);
+
+        cfg.setQueryConfiguration(qcfg);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSqlQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSqlQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSqlQueryMultiThreadedSelfTest.java
index 11ddf84..3a03ea8 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSqlQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSqlQueryMultiThreadedSelfTest.java
@@ -10,12 +10,11 @@
 package org.gridgain.grid.kernal.processors.cache;
 
 import org.apache.ignite.configuration.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.query.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.indexing.h2.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.query.*;
 import org.gridgain.testframework.*;
 import org.gridgain.testframework.junits.common.*;
 
@@ -44,10 +43,6 @@ public class GridCacheSqlQueryMultiThreadedSelfTest extends GridCommonAbstractTe
 
         c.setDiscoverySpi(disco);
 
-        GridH2IndexingSpi indexing = new GridH2IndexingSpi();
-
-        c.setIndexingSpi(indexing);
-
         GridCacheConfiguration ccfg = new GridCacheConfiguration();
 
         ccfg.setCacheMode(PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
index 1df68e9..c8adf83 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
@@ -14,13 +14,13 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.query.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.noop.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.junits.common.*;
 
@@ -29,11 +29,11 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.IgniteEventType.*;
 import static org.apache.ignite.configuration.IgniteDeploymentMode.*;
 import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.cache.GridCachePeekMode.*;
 import static org.gridgain.grid.cache.GridCacheWriteSynchronizationMode.*;
-import static org.apache.ignite.events.IgniteEventType.*;
 
 /**
  * Test for cache swap.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
index 9cdb1c7..ee404cf 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
@@ -11,15 +11,14 @@ package org.gridgain.grid.kernal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.eviction.fifo.*;
-import org.gridgain.grid.cache.query.*;
-import org.gridgain.grid.kernal.processors.cache.GridCacheAbstractQuerySelfTest.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.indexing.h2.*;
 import org.apache.ignite.spi.swapspace.noop.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.eviction.fifo.*;
+import org.gridgain.grid.cache.query.*;
+import org.gridgain.grid.kernal.processors.cache.GridCacheAbstractQuerySelfTest.*;
 import org.gridgain.testframework.junits.common.*;
 
 import java.util.*;
@@ -48,12 +47,6 @@ public class GridIndexingWithNoopSwapSelfTest extends GridCommonAbstractTest {
 
         c.setDiscoverySpi(disco);
 
-        GridH2IndexingSpi indexing = new GridH2IndexingSpi();
-
-        indexing.setDefaultIndexPrimitiveKey(true);
-
-        c.setIndexingSpi(indexing);
-
         c.setSwapSpaceSpi(new NoopSwapSpaceSpi());
 
         GridCacheConfiguration cc = defaultCacheConfiguration();
@@ -68,6 +61,12 @@ public class GridIndexingWithNoopSwapSelfTest extends GridCommonAbstractTest {
         cc.setBackups(1);
         cc.setAtomicityMode(TRANSACTIONAL);
 
+        GridCacheQueryConfiguration qcfg = new GridCacheQueryConfiguration();
+
+        qcfg.setIndexPrimitiveKey(true);
+
+        cc.setQueryConfiguration(qcfg);
+
         c.setCacheConfiguration(cc);
 
         return c;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedFieldsQuerySelfTest.java
index 6e80a55..43b0ad5 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedFieldsQuerySelfTest.java
@@ -17,8 +17,8 @@ import org.jetbrains.annotations.*;
 
 import java.util.*;
 
-import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
 
 /**
  * Tests for fields queries.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
index 022a2fd..7cd4d3b 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
@@ -12,18 +12,18 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.dataload.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.cache.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.indexing.h2.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.query.*;
 import org.gridgain.testframework.junits.common.*;
 
 import java.util.*;
 
-import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.cache.GridCacheWriteSynchronizationMode.*;
 
 /**
@@ -43,12 +43,6 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        // IndexingSpi
-        GridH2IndexingSpi spi = new GridH2IndexingSpi();
-        spi.setName("indexingSpi");
-        spi.setDefaultIndexPrimitiveKey(true);
-        cfg.setIndexingSpi(spi);
-
         // DiscoverySpi
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
         disco.setIpFinder(IP_FINDER);
@@ -85,6 +79,12 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac
         cfg.setPreloadPartitionedDelay(-1);
         cfg.setBackups(1);
 
+        GridCacheQueryConfiguration qcfg = new GridCacheQueryConfiguration();
+
+        qcfg.setIndexPrimitiveKey(true);
+
+        cfg.setQueryConfiguration(qcfg);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
index 890fe1d..b3e123f 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
@@ -12,14 +12,13 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.cache.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.indexing.h2.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
@@ -28,8 +27,8 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
-import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
 
 /**
  * Test for distributed queries with node restarts.
@@ -72,13 +71,13 @@ public class GridCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTest
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setDistributionMode(NEAR_PARTITIONED);
 
-        c.setCacheConfiguration(cc);
+        GridCacheQueryConfiguration qcfg = new GridCacheQueryConfiguration();
 
-        GridH2IndexingSpi idxSpi = new GridH2IndexingSpi();
+        qcfg.setIndexPrimitiveKey(true);
 
-        idxSpi.setDefaultIndexPrimitiveKey(true);
+        cc.setQueryConfiguration(qcfg);
 
-        c.setIndexingSpi(idxSpi);
+        c.setCacheConfiguration(cc);
 
         return c;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
index 81c1c95..5c2a3ef 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
@@ -13,12 +13,12 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.indexing.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.cache.*;
+import org.gridgain.grid.kernal.processors.query.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -27,8 +27,8 @@ import org.gridgain.testframework.*;
 import java.util.*;
 import java.util.concurrent.*;
 
-import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.apache.ignite.events.IgniteEventType.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
 
 /**
  * Tests for fields queries.
@@ -51,7 +51,7 @@ public class GridCacheReplicatedFieldsQuerySelfTest extends GridCacheAbstractFie
         hasCache = true;
 
         try {
-            final Map<UUID, Map<Long, GridFutureAdapter<IndexingFieldsResult>>> map =
+            final Map<UUID, Map<Long, GridFutureAdapter<GridQueryFieldsResult>>> map =
                 U.field(((GridKernal)grid(0)).internalCache().context().queries(), "fieldsQryRes");
 
             // Ensure that iterators map empty.
@@ -82,7 +82,7 @@ public class GridCacheReplicatedFieldsQuerySelfTest extends GridCacheAbstractFie
                 }
             }, getTestTimeout()));
 
-            Map<Long, GridFutureAdapter<IndexingFieldsResult>> futs = map.get(g.cluster().localNode().id());
+            Map<Long, GridFutureAdapter<GridQueryFieldsResult>> futs = map.get(g.cluster().localNode().id());
 
             assertEquals(1, futs.size());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
index c3da7e0..53d3dfd 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
@@ -13,18 +13,16 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.indexing.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.query.*;
+import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.future.*;
 import org.gridgain.testframework.*;
 import org.jetbrains.annotations.*;
 import org.springframework.util.*;
@@ -239,7 +237,7 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
      * @return {@code qryIters} of {@link GridCacheQueryManager}.
      */
     private ConcurrentMap<UUID,
-        Map<Long, GridFutureAdapter<GridCloseableIterator<IndexingKeyValueRow<CacheKey, CacheValue>>>>>
+        Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<CacheKey, CacheValue>>>>>
         distributedQueryManagerQueryItersMap(Ignite g) {
         GridCacheContext ctx = ((GridKernal)g).internalCache().context();
 
@@ -248,7 +246,7 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
         qryItersField.setAccessible(true);
 
         return (ConcurrentMap<UUID,
-            Map<Long, GridFutureAdapter<GridCloseableIterator<IndexingKeyValueRow<CacheKey, CacheValue>>>>>)
+            Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<CacheKey, CacheValue>>>>>)
             ReflectionUtils.getField(qryItersField, ctx.queries());
     }
 
@@ -397,7 +395,7 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
             assertEquals(0, (int)fut.next().getKey());
 
             final ConcurrentMap<UUID, Map<Long, GridFutureAdapter<GridCloseableIterator<
-                IndexingKeyValueRow<Integer, Integer>>>>> map =
+                IgniteBiTuple<Integer, Integer>>>>> map =
                 U.field(((GridKernal)grid(0)).internalCache().context().queries(), "qryIters");
 
             // fut.nextX() does not guarantee the request has completed on remote node
@@ -408,13 +406,13 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
                 }
             }, getTestTimeout()));
 
-            Map<Long, GridFutureAdapter<GridCloseableIterator<IndexingKeyValueRow<Integer, Integer>>>> futs =
+            Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<Integer, Integer>>>> futs =
                 map.get(g.cluster().localNode().id());
 
             assertEquals(1, futs.size());
 
-            IgniteSpiCloseableIterator<IndexingKeyValueRow<Integer, Integer>> iter =
-                U.field(((IgniteFuture)F.first(futs.values()).get()).get(), "iter");
+            GridCloseableIterator<IgniteBiTuple<Integer, Integer>> iter =
+                (GridCloseableIterator<IgniteBiTuple<Integer, Integer>>)((IgniteFuture)F.first(futs.values()).get()).get();
 
             ResultSet rs = U.field(iter, "data");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexRebuildTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexRebuildTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexRebuildTest.java
new file mode 100644
index 0000000..b288f0b
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexRebuildTest.java
@@ -0,0 +1,241 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.query.*;
+import org.gridgain.grid.kernal.processors.cache.*;
+import org.gridgain.grid.kernal.processors.query.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ */
+public class GridH2IndexRebuildTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int GRID_CNT = 1;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return GRID_CNT;
+    }
+
+    /**
+     * Overrides rebuildIndexes to check it can be interrupted.
+     */
+    private static class SleepingH2Indexing extends GridH2Indexing {
+        /** */
+        private volatile boolean sleepInRebuild;
+
+        /** */
+        private volatile boolean interrupted;
+
+        /**
+         * Constructor.
+         */
+        public SleepingH2Indexing() {
+            spi = this;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void rebuildIndexes(@Nullable String spaceName, GridQueryTypeDescriptor type) {
+            if (sleepInRebuild) {
+                try {
+                    U.sleep(Long.MAX_VALUE);
+                }
+                catch (GridInterruptedException ignored) {
+                    interrupted = true;
+                }
+            }
+
+            super.rebuildIndexes(spaceName, type);
+        }
+    }
+
+    /** */
+    private static SleepingH2Indexing spi;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        GridQueryProcessor.idxCls = SleepingH2Indexing.class;
+
+        return cfg;
+    }
+
+    /**
+     * Value class with regular and compound indexes.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestValue1 {
+        /** */
+        @GridCacheQuerySqlField(index = true)
+        private long val1;
+
+        /** */
+        @GridCacheQuerySqlField(index = true)
+        private String val2;
+
+        /** */
+        @GridCacheQuerySqlField(groups = "group1")
+        private int val3;
+
+        /** */
+        @GridCacheQuerySqlField(groups = "group1")
+        private int val4;
+
+        /**
+         */
+        TestValue1(long val1, String val2, int val3, int val4) {
+            this.val1 = val1;
+            this.val2 = val2;
+            this.val3 = val3;
+            this.val4 = val4;
+        }
+    }
+
+    /**
+     * Value class with regular and text indexes.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestValue2 {
+        /** */
+        @GridCacheQuerySqlField(index = true)
+        private long val1;
+
+        /** */
+        @GridCacheQueryTextField
+        private String val2;
+
+        /**
+         */
+        TestValue2(long val1, String val2) {
+            this.val1 = val1;
+            this.val2 = val2;
+        }
+    }
+
+    /** */
+    private static final int ENTRY_CNT = 10000;
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRebuildIndexes() throws Exception {
+        cache().queries().rebuildIndexes(ArrayList.class).get();
+
+        cache().queries().rebuildAllIndexes().get();
+
+        GridCache<Integer, TestValue1> cache1 = grid(0).cache(null);
+        GridCache<Integer, TestValue2> cache2 = grid(0).cache(null);
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            cache1.put(i, new TestValue1(i, "val2-" + i, i, i));
+            cache2.put(ENTRY_CNT * 2 + i, new TestValue2(i, "val2-" + i));
+        }
+
+        GridCacheQuery<Map.Entry<Integer, TestValue1>> qry1 =
+            cache1.queries().createSqlQuery(TestValue1.class, "val1 = 9000");
+
+        GridCacheQuery<Map.Entry<Integer, TestValue1>> qry2 =
+            cache1.queries().createSqlQuery(TestValue1.class, "val2 = 'val2-9000'");
+
+        GridCacheQuery<Map.Entry<Integer, TestValue1>> qry3 =
+            cache1.queries().createSqlQuery(TestValue1.class, "val3 = 9000 and val4 = 9000");
+
+        GridCacheQuery<Map.Entry<Integer, TestValue2>> qry4 =
+            cache2.queries().createSqlQuery(TestValue2.class, "val1 = 9000");
+
+        GridCacheQuery<Map.Entry<Integer, TestValue2>> qry5 =
+            cache2.queries().createFullTextQuery(TestValue2.class, "val2 = 'val2-9000'");
+
+        checkQueryReturnsOneEntry(qry1, qry2, qry3, qry4, qry5);
+
+        for (int i = 0; i < ENTRY_CNT / 2; i++) {
+            cache1.remove(i);
+            cache2.remove(ENTRY_CNT * 2 + i);
+        }
+
+        cache().queries().rebuildIndexes(TestValue1.class).get();
+        cache().queries().rebuildIndexes(TestValue2.class).get();
+
+        checkQueryReturnsOneEntry(qry1, qry2, qry3, qry4, qry5);
+
+        cache().queries().rebuildAllIndexes().get();
+
+        checkQueryReturnsOneEntry(qry1, qry2, qry3, qry4, qry5);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRebuildInterrupted() throws Exception {
+        spi.sleepInRebuild = true;
+
+        GridCache<Integer, TestValue1> cache1 = grid(0).cache(null);
+        GridCache<Integer, TestValue2> cache2 = grid(0).cache(null);
+
+        cache1.put(0, new TestValue1(0, "val0", 0 ,0));
+        cache2.put(1, new TestValue2(0, "val0"));
+
+        checkCancel(grid(0).cache(null).queries().rebuildIndexes("TestValue1"));
+
+        checkCancel(grid(0).cache(null).queries().rebuildAllIndexes());
+
+        spi.sleepInRebuild = false;
+
+        final IgniteFuture<?> fut1 = grid(0).cache(null).queries().rebuildIndexes(TestValue1.class);
+
+        assertFalse(fut1.isCancelled());
+
+        fut1.get();
+
+        final IgniteFuture<?> fut2 = grid(0).cache(null).queries().rebuildAllIndexes();
+
+        assertFalse(fut2.isCancelled());
+
+        fut2.get();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    private void checkCancel(final IgniteFuture<?> fut) throws Exception {
+        assertTrue(fut.cancel());
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                fut.get();
+                return null;
+            }
+        }, IgniteFutureCancelledException.class, null);
+
+        assertTrue(spi.interrupted);
+
+        spi.interrupted = false;
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    private void checkQueryReturnsOneEntry(GridCacheQuery<?>... qrys) throws Exception {
+        for (GridCacheQuery<?> qry : qrys)
+            assertEquals(1, qry.execute().get().size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingGeoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingGeoSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingGeoSelfTest.java
new file mode 100644
index 0000000..1f22529
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingGeoSelfTest.java
@@ -0,0 +1,240 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2;
+
+import com.vividsolutions.jts.geom.*;
+import com.vividsolutions.jts.io.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.query.*;
+import org.gridgain.grid.kernal.processors.cache.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class GridH2IndexingGeoSelfTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int CNT = 100;
+
+    /** */
+    private static final long DUR = 60000L;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return DUR * 3;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testGeo() throws Exception {
+        GridCache<Integer, EnemyCamp> cache = grid(0).cache(null);
+
+        WKTReader r = new WKTReader();
+
+        cache.put(0, new EnemyCamp(r.read("POINT(25 75)"), "A"));
+        cache.put(1, new EnemyCamp(r.read("POINT(70 70)"), "B"));
+        cache.put(2, new EnemyCamp(r.read("POINT(70 30)"), "C"));
+        cache.put(3, new EnemyCamp(r.read("POINT(75 25)"), "D"));
+
+        GridCacheQuery<Map.Entry<Integer, EnemyCamp>> qry = cache.queries().createSqlQuery(EnemyCamp.class,
+            "coords && ?");
+
+        Collection<Map.Entry<Integer, EnemyCamp>> res = qry.execute(r.read("POLYGON((5 70, 5 80, 30 80, 30 70, 5 70))"))
+            .get();
+
+        checkPoints(res, "A");
+
+        res = qry.execute(r.read("POLYGON((10 5, 10 35, 70 30, 75 25, 10 5))")).get();
+
+        checkPoints(res, "C", "D");
+
+        // Move B to the first polygon.
+        cache.put(1, new EnemyCamp(r.read("POINT(20 75)"), "B"));
+
+        res = qry.execute(r.read("POLYGON((5 70, 5 80, 30 80, 30 70, 5 70))")).get();
+
+        checkPoints(res, "A", "B");
+
+        // Move B to the second polygon.
+        cache.put(1, new EnemyCamp(r.read("POINT(30 30)"), "B"));
+
+        res = qry.execute(r.read("POLYGON((10 5, 10 35, 70 30, 75 25, 10 5))")).get();
+
+        checkPoints(res, "B", "C", "D");
+
+        // Remove B.
+        cache.remove(1);
+
+        res = qry.execute(r.read("POLYGON((5 70, 5 80, 30 80, 30 70, 5 70))")).get();
+
+        checkPoints(res, "A");
+
+        res = qry.execute(r.read("POLYGON((10 5, 10 35, 70 30, 75 25, 10 5))")).get();
+
+        checkPoints(res, "C", "D");
+
+        // Check explaint request.
+        assertTrue(F.first(cache.queries().createSqlFieldsQuery("explain select * from EnemyCamp " +
+            "where coords && 'POINT(25 75)'").execute().get()).get(0).toString().contains("coords_idx"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testGeoMultithreaded() throws Exception {
+        final GridCache<Integer, EnemyCamp> cache1 = grid(0).cache(null);
+        final GridCache<Integer, EnemyCamp> cache2 = grid(1).cache(null);
+        final GridCache<Integer, EnemyCamp> cache3 = grid(2).cache(null);
+
+        final String[] points = new String[CNT];
+
+        WKTReader r = new WKTReader();
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int idx = 0; idx < CNT; idx++) {
+            int x = rnd.nextInt(1, 100);
+            int y = rnd.nextInt(1, 100);
+
+            cache1.put(idx, new EnemyCamp(r.read("POINT(" + x + " " + y + ")"), Integer.toString(idx)));
+
+            points[idx] = Integer.toString(idx);
+        }
+
+        Thread.sleep(200);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+        final AtomicReference<Exception> err = new AtomicReference<>();
+
+        IgniteFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                WKTReader r = new WKTReader();
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                while (!stop.get()) {
+                    int cacheIdx = rnd.nextInt(0, 3);
+
+                    GridCache<Integer, EnemyCamp> cache = cacheIdx == 0 ? cache1 : cacheIdx == 1 ? cache2 : cache3;
+
+                    int idx = rnd.nextInt(CNT);
+                    int x = rnd.nextInt(1, 100);
+                    int y = rnd.nextInt(1, 100);
+
+                    cache.put(idx, new EnemyCamp(r.read("POINT(" + x + " " + y + ")"), Integer.toString(idx)));
+
+                    U.sleep(50);
+                }
+
+                return null;
+            }
+        }, Runtime.getRuntime().availableProcessors(), "put-thread");
+
+        IgniteFuture<?> qryFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                WKTReader r = new WKTReader();
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                while (!stop.get()) {
+                    try {
+                        int cacheIdx = rnd.nextInt(0, 3);
+
+                        GridCache<Integer, EnemyCamp> cache = cacheIdx == 0 ? cache1 : cacheIdx == 1 ? cache2 : cache3;
+
+                        GridCacheQuery<Map.Entry<Integer, EnemyCamp>> qry = cache.queries().createSqlQuery(
+                            EnemyCamp.class, "coords && ?");
+
+                        Collection<Map.Entry<Integer, EnemyCamp>> res = qry.execute(
+                            r.read("POLYGON((0 0, 0 100, 100 100, 100 0, 0 0))")).get();
+
+                        checkPoints(res, points);
+
+                        U.sleep(5);
+                    }
+                    catch (Exception e) {
+                        err.set(e);
+
+                        stop.set(true);
+
+                        break;
+                    }
+                }
+
+                return null;
+            }
+        }, 4, "qry-thread");
+
+        U.sleep(60000L);
+
+        stop.set(true);
+
+        putFut.get();
+        qryFut.get();
+
+        Exception err0 = err.get();
+
+        if (err0 != null)
+            throw err0;
+    }
+
+    /**
+     * Check whether result contains all required points.
+     *
+     * @param res Result.
+     * @param points Expected points.
+     */
+    private void checkPoints( Collection<Map.Entry<Integer, EnemyCamp>> res, String... points) {
+        Set<String> set = new HashSet<>(Arrays.asList(points));
+
+        assertEquals(set.size(), res.size());
+
+        for (Map.Entry<Integer, EnemyCamp> e : res)
+            assertTrue(set.remove(e.getValue().name));
+    }
+
+    /**
+     *
+     */
+    private static class EnemyCamp implements Serializable {
+        /** */
+        @GridCacheQuerySqlField(index = true)
+        private Geometry coords;
+
+        /** */
+        @GridCacheQuerySqlField
+        private String name;
+
+        /**
+         * @param coords Coordinates.
+         * @param name Name.
+         */
+        private EnemyCamp(Geometry coords, String name) {
+            this.coords = coords;
+            this.name = name;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingInMemSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingInMemSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingInMemSelfTest.java
new file mode 100644
index 0000000..ced0025
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingInMemSelfTest.java
@@ -0,0 +1,17 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2;
+
+/**
+ * Tests for H2 indexing SPI.
+ */
+public class GridH2IndexingInMemSelfTest extends GridIndexingSpiAbstractSelfTest {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingOffheapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingOffheapSelfTest.java
new file mode 100644
index 0000000..45aee5d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridH2IndexingOffheapSelfTest.java
@@ -0,0 +1,36 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2;
+
+/**
+ * Tests for H2 indexing SPI.
+ */
+public class GridH2IndexingOffheapSelfTest extends GridIndexingSpiAbstractSelfTest {
+    /** */
+    private static final long offheap = 10000000;
+
+    private static GridH2Indexing currentSpi;
+
+    /** {@inheritDoc} */
+    @Override protected void startIndexing(GridH2Indexing spi) throws Exception {
+        spi.configuration().setMaxOffHeapMemory(offheap);
+
+        currentSpi = spi;
+
+        super.startIndexing(spi);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        assertEquals(0, currentSpi.getAllocatedOffHeapMemory());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
new file mode 100644
index 0000000..135fd62
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -0,0 +1,553 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.kernal.processors.query.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.spi.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Tests for all SQL based indexing SPI implementations.
+ */
+public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TextIndex textIdx = new TextIndex(F.asList("txt"));
+
+    /** */
+    private static final Map<String, Class<?>> fieldsAA = new HashMap<>();
+
+    /** */
+    private static final Map<String, Class<?>> fieldsAB = new HashMap<>();
+
+    /** */
+    private static final Map<String, Class<?>> fieldsBA = new HashMap<>();
+
+    /**
+     * Fields initialization.
+     */
+    static {
+        fieldsAA.put("id", Long.class);
+        fieldsAA.put("name", String.class);
+        fieldsAA.put("age", Integer.class);
+
+        fieldsAB.putAll(fieldsAA);
+        fieldsAB.put("txt", String.class);
+
+        fieldsBA.putAll(fieldsAA);
+        fieldsBA.put("sex", Boolean.class);
+    }
+
+    /** */
+    private static TypeDesc typeAA = new TypeDesc("A", "A", fieldsAA, null);
+
+    /** */
+    private static TypeDesc typeAB = new TypeDesc("A", "B", fieldsAB, textIdx);
+
+    /** */
+    private static TypeDesc typeBA = new TypeDesc("B", "A", fieldsBA, null);
+
+    /** */
+    private GridH2Indexing idx = new GridH2Indexing();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        getTestResources().inject(idx);
+
+        startIndexing(idx);
+    }
+
+    /** {@inheritDoc} */
+    protected void startIndexing(GridH2Indexing spi) throws Exception {
+        spi.registerSpace("A");
+        spi.registerSpace("B");
+
+        spi.start(null);
+    }
+
+    @Override protected void afterTest() throws Exception {
+        idx.stop();
+
+        idx = null;
+    }
+
+    /**
+     * @param id Id.
+     * @param name Name.
+     * @param age Age.
+     * @return AA.
+     */
+    private Map<String, Object> aa(long id, String name, int age) {
+        Map<String, Object> map = new HashMap<>();
+
+        map.put("id", id);
+        map.put("name", name);
+        map.put("age", age);
+
+        return map;
+    }
+
+    /**
+     * @param id Id.
+     * @param name Name.
+     * @param age Age.
+     * @param txt Text.
+     * @return AB.
+     */
+    private Map<String, Object> ab(long id, String name, int age, String txt) {
+        Map<String, Object> map = aa(id, name, age);
+
+        map.put("txt", txt);
+
+        return map;
+    }
+
+    /**
+     * @param id Id.
+     * @param name Name.
+     * @param age Age.
+     * @param sex Sex.
+     * @return BA.
+     */
+    private Map<String, Object> ba(long id, String name, int age, boolean sex) {
+        Map<String, Object> map = aa(id, name, age);
+
+        map.put("sex", sex);
+
+        return map;
+    }
+
+    /**
+     * @param row Row
+     * @return Value.
+     * @throws IgniteSpiException If failed.
+     */
+    private Map<String, Object> value(IgniteBiTuple<Integer, Map<String, Object>> row) throws IgniteSpiException {
+        return row.get2();
+    }
+
+    /**
+     * @return Indexing.
+     */
+    private GridH2Indexing getIndexing() {
+        return idx;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSpi() throws Exception {
+        GridH2Indexing spi = getIndexing();
+
+        assertEquals(-1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(-1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(-1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.registerType(typeAA.space(), typeAA);
+
+        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(-1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(-1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.registerType(typeAB.space(), typeAB);
+
+        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(0, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(-1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.registerType(typeBA.space(), typeBA);
+
+        // Initially all is empty.
+        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(0, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+        assertFalse(spi.query(typeAA.space(), "select * from A.A", Collections.emptySet(), typeAA, null).hasNext());
+        assertFalse(spi.query(typeAB.space(), "select * from A.B", Collections.emptySet(), typeAB, null).hasNext());
+        assertFalse(spi.query(typeBA.space(), "select * from B.A", Collections.emptySet(), typeBA, null).hasNext());
+
+        // Nothing to remove.
+        spi.remove("A", 1);
+        spi.remove("B", 1);
+
+        spi.store(typeAA.space(), typeAA, 1, aa(1, "Vasya", 10), "v1".getBytes(), 0);
+
+        assertEquals(1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(0, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+        spi.store(typeAB.space(), typeAB, 1, ab(1, "Vasya", 20, "Some text about Vasya goes here."),
+            "v2".getBytes(), 0);
+
+        // In one space all keys must be unique.
+        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+        spi.store(typeBA.space(), typeBA, 1, ba(2, "Petya", 25, true), "v3".getBytes(), 0);
+
+        // No replacement because of different space.
+        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.store(typeBA.space(), typeBA, 1, ba(2, "Kolya", 25, true), "v4".getBytes(), 0);
+
+        // Replacement in the same table.
+        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.store(typeAA.space(), typeAA, 2, aa(2, "Valera", 19), "v5".getBytes(), 0);
+
+        assertEquals(1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.store(typeAA.space(), typeAA, 3, aa(3, "Borya", 18), "v6".getBytes(), 0);
+
+        assertEquals(2, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.store(typeAB.space(), typeAB, 4, ab(4, "Vitalya", 20, "Very Good guy"), "v7".getBytes(), 0);
+
+        assertEquals(2, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(2, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(1, spi.size(typeBA.space(), typeBA, null));
+
+        // Query data.
+        Iterator<IgniteBiTuple<Integer, Map<String, Object>>> res =
+            spi.query(typeAA.space(), "select * from a order by age", Collections.emptySet(), typeAA, null);
+
+        assertTrue(res.hasNext());
+        assertEquals(aa(3, "Borya", 18), value(res.next()));
+        assertTrue(res.hasNext());
+        assertEquals(aa(2, "Valera", 19), value(res.next()));
+        assertFalse(res.hasNext());
+
+        res = spi.query(typeAB.space(), "select * from b order by name", Collections.emptySet(), typeAB, null);
+
+        assertTrue(res.hasNext());
+        assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here."), value(res.next()));
+        assertTrue(res.hasNext());
+        assertEquals(ab(4, "Vitalya", 20, "Very Good guy"), value(res.next()));
+        assertFalse(res.hasNext());
+
+        res = spi.query(typeBA.space(), "select * from a", Collections.emptySet(), typeBA, null);
+
+        assertTrue(res.hasNext());
+        assertEquals(ba(2, "Kolya", 25, true), value(res.next()));
+        assertFalse(res.hasNext());
+
+        // Text queries
+        Iterator<IgniteBiTuple<Integer, Map<String, Object>>> txtRes = spi.queryText(typeAB.space(), "good",
+            typeAB, null);
+
+        assertTrue(txtRes.hasNext());
+        assertEquals(ab(4, "Vitalya", 20, "Very Good guy"), value(txtRes.next()));
+        assertFalse(txtRes.hasNext());
+
+        // Fields query
+        GridQueryFieldsResult fieldsRes =
+            spi.queryFields(null, "select a.a.name n1, a.a.age a1, b.a.name n2, " +
+            "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null);
+
+        String[] aliases = {"N1", "A1", "N2", "A2"};
+        Object[] vals = { "Valera", 19, "Kolya", 25};
+
+        assertTrue(fieldsRes.iterator().hasNext());
+
+        List<?> fields = fieldsRes.iterator().next();
+
+        assertEquals(4, fields.size());
+
+        int i = 0;
+
+        for (Object f : fields) {
+            assertEquals(aliases[i], fieldsRes.metaData().get(i).fieldName());
+            assertEquals(vals[i++], f);
+        }
+
+        assertFalse(fieldsRes.iterator().hasNext());
+
+        // Query on not existing table should not fail.
+        assertFalse(spi.queryFields(null, "select * from not_existing_table",
+            Collections.emptySet(), null).iterator().hasNext());
+
+        // Remove
+        spi.remove(typeAA.space(), 2);
+
+        assertEquals(1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(2, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(1, spi.size(typeBA.space(), typeBA, null));
+
+        spi.remove(typeBA.space(), 1);
+
+        assertEquals(1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(2, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+        boolean h2IdxOffheap = spi.configuration().getMaxOffHeapMemory() > 0;
+
+        // At the time of this writing index rebuilding is not supported for GridH2Indexing with off-heap storage.
+        if (!h2IdxOffheap) {
+            // Rebuild
+
+            spi.rebuildIndexes(typeAB.space(), typeAB);
+
+            assertEquals(1, spi.size(typeAA.space(), typeAA, null));
+            assertEquals(2, spi.size(typeAB.space(), typeAB, null));
+            assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+            // For invalid space name/type should not fail.
+            spi.rebuildIndexes("not_existing_space", typeAA);
+            spi.rebuildIndexes(typeAA.space(), new TypeDesc("C", "C", fieldsAA, null));
+        }
+
+        // Unregister.
+        spi.unregisterType(typeAA.space(), typeAA);
+
+        assertEquals(-1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(2, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+        spi.unregisterType(typeAB.space(), typeAB);
+
+        assertEquals(-1, spi.size(typeAA.space(), typeAA, null));
+        assertEquals(-1, spi.size(typeAB.space(), typeAB, null));
+        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
+
+        spi.unregisterType(typeBA.space(), typeBA);
+
+        // Should not store but should not fail as well.
+        spi.store(typeAA.space(), typeAA, 10, aa(1, "Fail", 100500), "v220".getBytes(), 0);
+
+        assertEquals(-1, spi.size(typeAA.space(), typeAA, null));
+    }
+
+    /**
+     * Test long queries write explain warnings into log.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLongQueries() throws Exception {
+        GridH2Indexing spi = getIndexing();
+
+        long longQryExecTime = 100;
+
+        GridStringLogger log = new GridStringLogger(false, this.log);
+
+        IgniteLogger oldLog = GridTestUtils.getFieldValue(spi, "log");
+
+        spi.configuration().setLongQueryExecutionTimeout(longQryExecTime);
+        spi.configuration().setLongQueryExplain(true);
+
+        try {
+            GridTestUtils.setFieldValue(spi, "log", log);
+
+            String sql = "select sum(x) FROM SYSTEM_RANGE(?, ?)";
+
+            long now = U.currentTimeMillis();
+            long time = now;
+
+            long range = 1000000L;
+
+            while (now - time <= longQryExecTime * 3 / 2) {
+                time = now;
+                range *= 3;
+
+                GridQueryFieldsResult res = spi.queryFields(null, sql, Arrays.<Object>asList(1, range), null);
+
+                assert res.iterator().hasNext();
+
+                now = U.currentTimeMillis();
+            }
+
+            String res = log.toString();
+
+            F.println(res);
+
+            assert res.contains("/* PUBLIC.RANGE_INDEX */");
+        }
+        finally {
+            GridTestUtils.setFieldValue(spi, "log", oldLog);
+            spi.configuration().setLongQueryExecutionTimeout(3000);
+        }
+    }
+
+    public void _testResultReuse() throws Exception {
+        final GridH2Indexing spi = getIndexing();
+
+        multithreaded(new Callable<Object>() {
+              @Override public Object call() throws Exception {
+                  return spi.queryFields(null, "SELECT sum(x) + sum(x) + sum(x) + sum(x) FROM SYSTEM_RANGE(?, ?)",
+                      F.<Object>asList(0, 7000000), null);
+              }
+          }, 5);
+    }
+
+    /**
+     * Test long queries write explain warnings into log.
+     *
+     * @throws Exception If failed.
+     */
+    public void testZeroLongQuery() throws Exception {
+        GridH2Indexing spi = getIndexing();
+
+        long longQryExecTime = -1;
+
+        GridStringLogger log = new GridStringLogger(false, this.log);
+
+        IgniteLogger oldLog = GridTestUtils.getFieldValue(spi, "log");
+        spi.configuration().setLongQueryExecutionTimeout(longQryExecTime);
+        spi.configuration().setLongQueryExplain(true);
+
+        try {
+            GridTestUtils.setFieldValue(spi, "log", log);
+
+            String sql = "SELECT * FROM MyNonExistingTable";
+
+            GridQueryFieldsResult res = spi.queryFields(null, sql, Collections.emptyList(), null);
+
+            assertFalse(res.iterator().hasNext());
+
+            String logStr = log.toString();
+
+            F.println(logStr);
+
+            assertTrue(logStr.contains("Failed to explain plan because required table does not exist"));
+        }
+        finally {
+            GridTestUtils.setFieldValue(spi, "log", oldLog);
+            spi.configuration().setLongQueryExecutionTimeout(3000);
+        }
+    }
+
+    /**
+     * Index descriptor.
+     */
+    private static class TextIndex implements GridQueryIndexDescriptor {
+        /** */
+        private final Collection<String> fields;
+
+        /**
+         * @param fields Fields.
+         */
+        private TextIndex(Collection<String> fields) {
+            this.fields = Collections.unmodifiableCollection(fields);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<String> fields() {
+            return fields;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean descending(String field) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridQueryIndexType type() {
+            return GridQueryIndexType.FULLTEXT;
+        }
+    }
+
+    /**
+     * Type descriptor.
+     */
+    private static class TypeDesc implements GridQueryTypeDescriptor {
+        /** */
+        private final String name;
+
+        /** */
+        private final String space;
+
+        /** */
+        private final Map<String, Class<?>> valFields;
+
+        /** */
+        private final GridQueryIndexDescriptor textIdx;
+
+        /**
+         * @param space Space name.
+         * @param name Type name.
+         * @param valFields Fields.
+         * @param textIdx Fulltext index.
+         */
+        private TypeDesc(String space, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
+            this.name = name;
+            this.space = space;
+            this.valFields = Collections.unmodifiableMap(valFields);
+            this.textIdx = textIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String name() {
+            return name;
+        }
+
+        /**
+         * @return Space name.
+         */
+        public String space() {
+            return space;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<String, Class<?>> valueFields() {
+            return valFields;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<String, Class<?>> keyFields() {
+            return Collections.emptyMap();
+        }
+
+        /** {@inheritDoc} */
+        @Override public <T> T value(Object obj, String field) throws IgniteSpiException {
+            assert obj != null;
+            assert !F.isEmpty(field);
+
+            return (T)((Map<String, Object>) obj).get(field);
+        }
+
+        /** */
+        @Override public Map<String, GridQueryIndexDescriptor> indexes() {
+            return textIdx == null ? Collections.<String, GridQueryIndexDescriptor>emptyMap() :
+                Collections.singletonMap("index", textIdx);
+        }
+
+        /** */
+        @Override public Class<?> valueClass() {
+            return Object.class;
+        }
+
+        /** */
+        @Override public Class<?> keyClass() {
+            return Integer.class;
+        }
+
+        /** */
+        @Override public boolean valueTextIndex() {
+            return textIdx == null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/922a2bab/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/opt/GridH2TableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/opt/GridH2TableSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/opt/GridH2TableSelfTest.java
new file mode 100644
index 0000000..6d2f75c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/opt/GridH2TableSelfTest.java
@@ -0,0 +1,613 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.opt;
+
+import org.gridgain.testframework.junits.common.*;
+import org.h2.Driver;
+import org.h2.index.*;
+import org.h2.result.*;
+import org.h2.table.*;
+import org.h2.value.*;
+import org.jetbrains.annotations.*;
+import org.junit.*;
+
+import java.sql.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Tests H2 Table.
+ */
+@SuppressWarnings({"TypeMayBeWeakened", "FieldAccessedSynchronizedAndUnsynchronized"})
+public class GridH2TableSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final long MAX_X = 2000;
+
+    /** */
+    private static final String DB_URL = "jdbc:h2:mem:gg_table_engine;MULTI_THREADED=1;OPTIMIZE_REUSE_RESULTS=0;" +
+        "QUERY_CACHE_SIZE=0;RECOMPILE_ALWAYS=1";
+
+    /** */
+    private static final String CREATE_TABLE_SQL = "CREATE TABLE T(ID UUID, T TIMESTAMP, STR VARCHAR, X BIGINT)";
+
+    /** */
+    private static final String PK_NAME = "__GG_PK_";
+
+    /** */
+    private static final String STR_IDX_NAME = "__GG_IDX_";
+
+    /** */
+    private static final String NON_UNIQUE_IDX_NAME = "__GG_IDX_";
+
+    /** */
+    private static final String SCAN_IDX_NAME = GridH2Table.ScanIndex.SCAN_INDEX_NAME_SUFFIX;
+
+    /** */
+    private Connection conn;
+
+    /** */
+    private GridH2Table tbl;
+
+    /** {@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 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 GridH2TreeIndex(PK_NAME, tbl, true, 0, 1, id));
+                idxs.add(new GridH2TreeIndex(NON_UNIQUE_IDX_NAME, tbl, false, 0, 1, x, t));
+                idxs.add(new GridH2TreeIndex(STR_IDX_NAME, tbl, false, 0, 1, str));
+
+                return idxs;
+            }
+        }, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        conn.close();
+
+        conn = null;
+        tbl = null;
+    }
+
+    /**
+     * @param id Id.
+     * @param t Timestamp.
+     * @param str String.
+     * @param x X.
+     * @return New row.
+     */
+    private GridH2Row row(UUID id, long t, String str, long x) {
+        return new GridH2Row(ValueUuid.get(id.getMostSignificantBits(), id.getLeastSignificantBits()),
+            ValueTimestamp.get(new Timestamp(t)),
+            ValueString.get(str),
+            ValueLong.get(x));
+    }
+
+
+    /**
+     * Simple table test.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTable() throws Exception {
+        // Test insert.
+        long x = MAX_X;
+
+        Random rnd = new Random();
+
+        while(x-- > 0) {
+            UUID id = UUID.randomUUID();
+
+            GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() :
+                UUID.randomUUID().toString(), rnd.nextInt(100));
+
+            tbl.doUpdate(row, false);
+        }
+
+        assertEquals(MAX_X, tbl.getRowCountApproximation());
+        assertEquals(MAX_X, tbl.getRowCount(null));
+
+        for (GridH2IndexBase idx : tbl.indexes()) {
+            assertEquals(MAX_X, idx.getRowCountApproximation());
+            assertEquals(MAX_X, idx.getRowCount(null));
+        }
+
+        // Check correct rows order.
+        checkOrdered((GridH2TreeIndex)tbl.indexes().get(0), new Comparator<SearchRow>() {
+            @Override public int compare(SearchRow o1, SearchRow o2) {
+                UUID id1 = (UUID)o1.getValue(0).getObject();
+                UUID id2 = (UUID)o2.getValue(0).getObject();
+
+                return id1.compareTo(id2);
+            }
+        });
+
+        checkOrdered((GridH2TreeIndex)tbl.indexes().get(1), new Comparator<SearchRow>() {
+            @Override public int compare(SearchRow o1, SearchRow o2) {
+                Long x1 = (Long)o1.getValue(3).getObject();
+                Long x2 = (Long)o2.getValue(3).getObject();
+
+                int c = x2.compareTo(x1);
+
+                if (c != 0)
+                    return c;
+
+                Timestamp t1 = (Timestamp)o1.getValue(1).getObject();
+                Timestamp t2 = (Timestamp)o2.getValue(1).getObject();
+
+                return t1.compareTo(t2);
+            }
+        });
+
+        checkOrdered((GridH2TreeIndex)tbl.indexes().get(2), new Comparator<SearchRow>() {
+            @Override public int compare(SearchRow o1, SearchRow o2) {
+                String s1 = (String)o1.getValue(2).getObject();
+                String s2 = (String)o2.getValue(2).getObject();
+
+                return s2.compareTo(s1);
+            }
+        });
+
+        // Indexes data consistency.
+        ArrayList<? extends Index> idxs = tbl.indexes();
+
+        checkIndexesConsistent((ArrayList<Index>)idxs, null);
+
+        // Check unique index.
+        UUID id = UUID.randomUUID();
+        UUID id2 = UUID.randomUUID();
+
+        assertTrue(tbl.doUpdate(row(id, System.currentTimeMillis(), id.toString(), rnd.nextInt(100)), false));
+        assertTrue(tbl.doUpdate(row(id2, System.currentTimeMillis(), id2.toString(), rnd.nextInt(100)), false));
+
+        // Check index selection.
+        checkQueryPlan(conn, "SELECT * FROM T", SCAN_IDX_NAME);
+
+        checkQueryPlan(conn, "SELECT * FROM T WHERE ID IS NULL", PK_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T WHERE ID = RANDOM_UUID()", PK_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T WHERE ID > RANDOM_UUID()", PK_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T ORDER BY ID", PK_NAME);
+
+        checkQueryPlan(conn, "SELECT * FROM T WHERE STR IS NULL", STR_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T WHERE STR = 'aaaa'", STR_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T WHERE STR > 'aaaa'", STR_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T ORDER BY STR DESC", STR_IDX_NAME);
+
+        checkQueryPlan(conn, "SELECT * FROM T WHERE X IS NULL", NON_UNIQUE_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T WHERE X = 10000", NON_UNIQUE_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T WHERE X > 10000", NON_UNIQUE_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC", NON_UNIQUE_IDX_NAME);
+        checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC, T", NON_UNIQUE_IDX_NAME);
+
+        checkQueryPlan(conn, "SELECT * FROM T ORDER BY T, X DESC", SCAN_IDX_NAME);
+
+        // Simple queries.
+
+        Statement s = conn.createStatement();
+
+        ResultSet rs = s.executeQuery("select id from t where x between 0 and 100");
+
+        int i = 0;
+        while (rs.next())
+            i++;
+
+        assertEquals(MAX_X + 2, i);
+
+        // -----
+
+        rs = s.executeQuery("select id from t where t is not null");
+
+        i = 0;
+        while (rs.next())
+            i++;
+
+        assertEquals(MAX_X + 2, i);
+
+        // ----
+
+        int cnt = 10 + rnd.nextInt(25);
+
+        long t = System.currentTimeMillis();
+
+        for (i = 0; i < cnt; i++) {
+            id = UUID.randomUUID();
+
+            assertTrue(tbl.doUpdate(row(id, t, id.toString(), 51), false));
+        }
+
+        rs = s.executeQuery("select x, id from t where x = 51 limit " + cnt);
+
+        i = 0;
+
+        while (rs.next()) {
+            assertEquals(51, rs.getInt(1));
+
+            i++;
+        }
+
+        assertEquals(cnt, i);
+    }
+
+    /**
+     * Dumps all table rows for index.
+     *
+     * @param idx Index.
+     */
+    private void dumpRows(GridH2TreeIndex idx) {
+        Iterator<GridH2Row> iter = idx.rows();
+
+        while (iter.hasNext())
+            System.out.println(iter.next().toString());
+    }
+
+    /**
+     * Multithreaded indexes consistency test.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIndexesMultiThreadedConsistency() throws Exception {
+        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 {
+        int rows = 3000;
+        int xs = 37;
+
+        long t = System.currentTimeMillis();
+
+        Random rnd = new Random();
+
+        for (int i = 0 ; i < rows; i++) {
+            UUID id = UUID.randomUUID();
+
+            GridH2Row row = row(id, t++, id.toString(), rnd.nextInt(xs));
+
+            assertTrue(tbl.doUpdate(row, false));
+        }
+
+        PreparedStatement ps = conn.prepareStatement("select count(*) from t where x = ?");
+
+        int cnt = 0;
+
+        for (int x = 0; x < xs; x++) {
+            ps.setInt(1, x);
+
+            ResultSet rs = ps.executeQuery();
+
+            assertTrue(rs.next());
+
+            cnt += rs.getInt(1);
+        }
+
+        assertEquals(rows, cnt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDataLoss() throws Exception {
+        final int threads = 37;
+        final int iterations = 15000;
+
+        final AtomicInteger cntr = new AtomicInteger();
+
+        final UUID[] ids = new UUID[threads * iterations];
+
+        for (int i = 0; i < ids.length; i++)
+            ids[i] = UUID.randomUUID();
+
+        final long t = System.currentTimeMillis();
+
+        final AtomicInteger deleted = new AtomicInteger();
+
+        multithreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Random rnd = new Random();
+
+                int offset = cntr.getAndIncrement() * iterations;
+
+                synchronized (ids[offset]) {
+                    for (int i = 0; i < iterations; i++) {
+                        UUID id = ids[offset + i];
+
+                        int x = rnd.nextInt(50);
+
+                        GridH2Row row = row(id, t, id.toString(), x);
+
+                        assertTrue(tbl.doUpdate(row, false));
+                    }
+                }
+
+                offset = (offset + iterations) % ids.length;
+
+                synchronized (ids[offset]) {
+                    for (int i = 0; i < iterations; i += 2) {
+                        UUID id = ids[offset + i];
+
+                        int x = rnd.nextInt(50);
+
+                        GridH2Row row = row(id, t, id.toString(), x);
+
+                        if (tbl.doUpdate(row, true))
+                            deleted.incrementAndGet();
+                    }
+                }
+
+                return null;
+            }
+        }, threads);
+
+        assertTrue(deleted.get() > 0);
+
+        PreparedStatement p = conn.prepareStatement("select count(*) from t where id = ?");
+
+        for (int i = 1; i < ids.length; i += 2) {
+            p.setObject(1, ids[i]);
+
+            ResultSet rs = p.executeQuery();
+
+            assertTrue(rs.next());
+
+            assertEquals(1, rs.getInt(1));
+        }
+
+        Statement s = conn.createStatement();
+
+        ResultSet rs = s.executeQuery("select count(*) from t");
+
+        assertTrue(rs.next());
+
+        assertEquals(ids.length - deleted.get(), rs.getInt(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRebuildIndexes() throws Exception {
+        ArrayList<GridH2IndexBase> idxsBefore = tbl.indexes();
+
+        assertEquals(3, idxsBefore.size());
+
+        Random rnd = new Random();
+
+        for (int i = 0; i < MAX_X; i++) {
+            UUID id = UUID.randomUUID();
+
+            GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() :
+                    UUID.randomUUID().toString(), rnd.nextInt(100));
+
+            tbl.doUpdate(row, false);
+        }
+
+        for (GridH2IndexBase idx : idxsBefore)
+            assertEquals(MAX_X, idx.getRowCountApproximation());
+
+        tbl.rebuildIndexes();
+
+        ArrayList<GridH2IndexBase> idxsAfter = tbl.indexes();
+
+        assertEquals(3, idxsAfter.size());
+
+        for (int i = 0; i < 3; i++) {
+            GridH2IndexBase idxBefore = idxsBefore.get(i);
+            GridH2IndexBase idxAfter = idxsAfter.get(i);
+
+            assertNotSame(idxBefore, idxAfter);
+            assertEquals(idxBefore.getName(), idxAfter.getName());
+            assertSame(idxBefore.getTable(), idxAfter.getTable());
+            assertEquals(idxBefore.getRowCountApproximation(), idxAfter.getRowCountApproximation());
+            assertEquals(idxBefore.getIndexType().isUnique(), idxAfter.getIndexType().isUnique());
+            Assert.assertArrayEquals(idxBefore.getColumns(), idxAfter.getColumns());
+        }
+    }
+
+    /**
+     * Check query plan to correctly select index.
+     *
+     * @param conn Connection.
+     * @param sql Select.
+     * @param search Search token in result.
+     * @throws SQLException If failed.
+     */
+    private void checkQueryPlan(Connection conn, String sql, String search) throws SQLException {
+
+        try (Statement s = conn.createStatement()) {
+            try (ResultSet r = s.executeQuery("EXPLAIN ANALYZE " + sql)) {
+                assertTrue(r.next());
+
+                String plan = r.getString(1);
+
+                assertTrue("Execution plan for '" + sql + "' query should contain '" + search + "'",
+                        plan.contains(search));
+            }
+        }
+    }
+
+    /**
+     * @param idxs Indexes.
+     * @param rowSet Rows.
+     * @return Rows.
+     */
+    private Set<Row> checkIndexesConsistent(ArrayList<Index> idxs, @Nullable Set<Row> rowSet) {
+        for (Index idx : idxs) {
+            if (!(idx instanceof GridH2TreeIndex))
+                continue;
+
+            Set<Row> set = new HashSet<>();
+
+            Iterator<GridH2Row> iter = ((GridH2TreeIndex)idx).rows();
+
+            while(iter.hasNext())
+                assertTrue(set.add(iter.next()));
+
+            //((GridH2SnapTreeSet)((GridH2Index)idx).tree).print();
+
+            if (rowSet == null)
+                rowSet = set;
+            else
+                assertEquals(rowSet, set);
+        }
+
+        return rowSet;
+    }
+
+    /**
+     * @param idxs Indexes list.
+     */
+    private void checkOrdered(ArrayList<Index> idxs) {
+        for (Index idx : idxs) {
+            if (!(idx instanceof GridH2TreeIndex))
+                continue;
+
+            GridH2TreeIndex h2Idx = (GridH2TreeIndex)idx;
+
+            checkOrdered(h2Idx, h2Idx);
+        }
+    }
+
+    /**
+     * @param idx Index.
+     * @param cmp Comparator.
+     */
+    private void checkOrdered(GridH2TreeIndex idx, Comparator<? super GridH2Row> cmp) {
+        Iterator<GridH2Row> rows = idx.rows();
+
+        GridH2Row min = null;
+
+        while (rows.hasNext()) {
+            GridH2Row row = rows.next();
+
+            assertNotNull(row);
+
+            assertFalse("Incorrect row order in index: " + idx + "\n min: " + min + "\n row: " + row,
+                min != null && cmp.compare(min, row) > 0);
+
+            min = row;
+        }
+    }
+}