You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by na...@apache.org on 2021/11/09 09:30:56 UTC

[ignite] 01/02: IGNITE-15530 IndexQuery uses MergeSort (#9540)

This is an automated email from the ASF dual-hosted git repository.

namelchev pushed a commit to branch ignite-2.12
in repository https://gitbox.apache.org/repos/asf/ignite.git

commit c39c56bad99c95166446191fe4c32866b5ad007e
Author: Maksim Timonin <ti...@gmail.com>
AuthorDate: Tue Nov 9 12:02:35 2021 +0300

    IGNITE-15530 IndexQuery uses MergeSort (#9540)
    
    (cherry picked from commit 19ac10dc49ca56b549b0cc728959ba49334d1ae9)
---
 .../cache/query/index/IndexQueryProcessor.java     | 215 ++++++++------
 .../BooleanIndexKey.java => IndexQueryResult.java} |  40 ++-
 .../cache/query/index/IndexQueryResultMeta.java    |  90 ++++++
 .../query/index/sorted/IndexKeyDefinition.java     |  35 ++-
 .../query/index/sorted/IndexKeyTypeSettings.java   |  26 +-
 .../query/index/sorted/IndexRowCompartorImpl.java  |   8 -
 .../query/index/sorted/keys/BooleanIndexKey.java   |   4 +-
 .../query/index/sorted/keys/ByteIndexKey.java      |   4 +-
 .../query/index/sorted/keys/BytesIndexKey.java     |   4 +-
 .../query/index/sorted/keys/DoubleIndexKey.java    |   4 +-
 .../query/index/sorted/keys/FloatIndexKey.java     |   4 +-
 .../query/index/sorted/keys/IntegerIndexKey.java   |   4 +-
 .../query/index/sorted/keys/LongIndexKey.java      |   4 +-
 .../query/index/sorted/keys/ShortIndexKey.java     |   4 +-
 .../index/sorted/keys/SignedBytesIndexKey.java     |   4 +-
 .../GridCacheDistributedFieldsQueryFuture.java     |   4 +-
 .../query/GridCacheDistributedQueryFuture.java     |  31 ++-
 .../query/GridCacheDistributedQueryManager.java    |  30 +-
 .../query/GridCacheLocalFieldsQueryFuture.java     |   4 +-
 .../cache/query/GridCacheLocalQueryManager.java    |   8 +-
 .../cache/query/GridCacheQueryFutureAdapter.java   |  32 ++-
 .../cache/query/GridCacheQueryManager.java         |  73 ++++-
 .../cache/query/GridCacheQueryResponse.java        |  42 ++-
 .../cache/query/reducer/CacheQueryReducer.java     |   7 +-
 .../cache/query/reducer/IndexQueryReducer.java     | 149 ++++++++++
 .../query/reducer/MergeSortCacheQueryReducer.java  |  20 +-
 .../cache/query/reducer/TextQueryReducer.java      |  47 ++++
 .../processors/query/GridQueryProcessor.java       |   7 +-
 .../processors/query/h2/index/H2RowComparator.java |   7 +-
 .../ignite/cache/query/IndexQueryAliasTest.java    |  30 +-
 .../ignite/cache/query/IndexQueryAllTypesTest.java |  25 +-
 .../ignite/cache/query/IndexQueryFilterTest.java   |  89 ++++--
 .../cache/query/IndexQueryKeepBinaryTest.java      |  52 +++-
 .../cache/query/IndexQueryQueryEntityTest.java     |  19 +-
 .../ignite/cache/query/IndexQueryRangeTest.java    | 246 ++++++----------
 .../ignite/cache/query/IndexQuerySqlIndexTest.java |  35 ++-
 .../ignite/cache/query/MultiTableIndexQuery.java   |   4 +-
 .../cache/query/MultifieldIndexQueryTest.java      | 309 +++++++--------------
 .../cache/query/RepeatedFieldIndexQueryTest.java   |  10 +-
 .../cache/GridCacheFullTextQueryPagesTest.java     |  11 +-
 40 files changed, 1057 insertions(+), 684 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
index bd9496b..a88a45d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
@@ -21,6 +21,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.PriorityQueue;
@@ -38,17 +40,18 @@ import org.apache.ignite.internal.cache.query.index.sorted.IndexRowComparator;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexSearchRowImpl;
 import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
 import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedSegmentedIndex;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexQueryContext;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
+import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.CacheObjectUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.query.IndexQueryDesc;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
-import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
@@ -70,20 +73,30 @@ public class IndexQueryProcessor {
         this.idxProc = idxProc;
     }
 
-    /** Run query on local node. */
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+    /**
+     * Run query on local node.
+     *
+     * @return Query result that contains data iterator and related metadata.
+     */
+    public <K, V> IndexQueryResult<K, V> queryLocal(
         GridCacheContext<K, V> cctx,
         IndexQueryDesc idxQryDesc,
         @Nullable IgniteBiPredicate<K, V> filter,
         IndexQueryContext qryCtx,
         boolean keepBinary
     ) throws IgniteCheckedException {
-        Index idx = index(cctx, idxQryDesc);
+        SortedSegmentedIndex idx = findSortedIndex(cctx, idxQryDesc);
+
+        IndexRangeQuery qry = prepareQuery(idx, idxQryDesc);
 
-        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc, qryCtx);
+        GridCursor<IndexRow> cursor = querySortedIndex(cctx, idx, qryCtx, qry);
+
+        SortedIndexDefinition def = (SortedIndexDefinition)idxProc.indexDefinition(idx.id());
+
+        IndexQueryResultMeta meta = new IndexQueryResultMeta(def, qry.criteria.length);
 
         // Map IndexRow to Cache Key-Value pair.
-        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+        return new IndexQueryResult<>(meta, new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
             private IgniteBiTuple<K, V> currVal;
 
             private final CacheObjectContext coctx = cctx.cacheObjectContext();
@@ -96,11 +109,16 @@ public class IndexQueryProcessor {
                 while (currVal == null && cursor.next()) {
                     IndexRow r = cursor.get();
 
-                    K k = (K)CacheObjectUtils.unwrapBinaryIfNeeded(coctx, r.cacheDataRow().key(), keepBinary, false);
-                    V v = (V)CacheObjectUtils.unwrapBinaryIfNeeded(coctx, r.cacheDataRow().value(), keepBinary, false);
+                    K k = unwrap(r.cacheDataRow().key(), true);
+                    V v = unwrap(r.cacheDataRow().value(), true);
 
-                    if (filter != null && !filter.apply(k, v))
-                        continue;
+                    if (filter != null) {
+                        K k0 = keepBinary ? k : unwrap(r.cacheDataRow().key(), false);
+                        V v0 = keepBinary ? v : unwrap(r.cacheDataRow().value(), false);
+
+                        if (!filter.apply(k0, v0))
+                            continue;
+                    }
 
                     currVal = new IgniteBiTuple<>(k, v);
                 }
@@ -120,16 +138,21 @@ public class IndexQueryProcessor {
 
                 return row;
             }
-        };
+
+            /** */
+            private <T> T unwrap(CacheObject o, boolean keepBinary) {
+                return (T)CacheObjectUtils.unwrapBinaryIfNeeded(coctx, o, keepBinary, false);
+            }
+        });
     }
 
     /**
-     * Finds index to run query and validates that criteria fields match a prefix of fields.
+     * Finds sorted index to run query by specified description.
      *
      * @return Index to run query by specified description.
      * @throws IgniteCheckedException If index not found.
      */
-    private Index index(GridCacheContext<?, ?> cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+    private SortedSegmentedIndex findSortedIndex(GridCacheContext<?, ?> cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
         final String tableName = cctx.kernalContext().query().tableName(cctx.name(), idxQryDesc.valType());
 
         if (tableName == null)
@@ -158,56 +181,38 @@ public class IndexQueryProcessor {
             return indexByCriteria(cctx, critFlds, tableName, idxQryDesc);
 
         // If index name isn't specified and criteria aren't set then use the PK index.
-        String idxName = idxQryDesc.idxName() == null ? QueryUtils.PRIMARY_KEY_INDEX : idxQryDesc.idxName();
+        String name = idxQryDesc.idxName() == null ? QueryUtils.PRIMARY_KEY_INDEX : idxQryDesc.idxName();
 
-        return indexByName(cctx, idxName, tableName, idxQryDesc, critFlds);
+        IndexName idxName = new IndexName(cctx.name(), cctx.kernalContext().query().schemaName(cctx), tableName, name);
+
+        return indexByName(idxName, idxQryDesc, critFlds);
     }
 
     /**
-     * @return Index found by name.
-     * @throws IgniteCheckedException If index not found.
+     * @return Sorted index found by name.
+     * @throws IgniteCheckedException If index not found or specified index doesn't match query criteria.
      */
-    private Index indexByName(
-        GridCacheContext<?, ?> cctx,
-        String idxName,
-        String tableName,
+    private SortedSegmentedIndex indexByName(
+        IndexName idxName,
         IndexQueryDesc idxQryDesc,
         final Map<String, String> criteriaFlds
     ) throws IgniteCheckedException {
-        String schema = cctx.kernalContext().query().schemaName(cctx);
-
-        IndexName name = new IndexName(cctx.name(), schema, tableName, idxName);
-
-        Index idx = getAndValidateIndex(name, idxQryDesc, criteriaFlds, false);
-
-        if (idx != null)
-            return idx;
+        SortedSegmentedIndex idx = assertSortedIndex(idxProc.index(idxName), idxQryDesc);
 
-        String normIdxName = idxName;
+        if (idx == null && !QueryUtils.PRIMARY_KEY_INDEX.equals(idxName.idxName())) {
+            String normIdxName = QueryUtils.normalizeObjectName(idxName.idxName(), false);
 
-        if (!QueryUtils.PRIMARY_KEY_INDEX.equals(idxName))
-            normIdxName = QueryUtils.normalizeObjectName(idxName, false);
+            idxName = new IndexName(idxName.cacheName(), idxName.schemaName(), idxName.tableName(), normIdxName);
 
-        name = new IndexName(cctx.name(), schema, tableName, normIdxName);
+            idx = assertSortedIndex(idxProc.index(idxName), idxQryDesc);
+        }
 
-        return getAndValidateIndex(name, idxQryDesc, criteriaFlds, true);
-    }
+        if (idx == null)
+            throw failIndexQuery("No index found for name: " + idxName.idxName(), null, idxQryDesc);
 
-    /** */
-    private @Nullable Index getAndValidateIndex(
-        IndexName name,
-        IndexQueryDesc idxQryDesc,
-        final Map<String, String> critFlds,
-        boolean failOnNotFound
-    ) throws IgniteCheckedException {
-        Index idx = idxProc.index(name);
-
-        if (idx != null && !critFlds.isEmpty() && !checkIndex(idxProc.indexDefinition(idx.id()), critFlds))
+        if (!checkIndex(idx, idxName.tableName(), criteriaFlds))
             throw failIndexQuery("Index doesn't match criteria", null, idxQryDesc);
 
-        if (idx == null && failOnNotFound)
-            throw failIndexQuery("No index found for name: " + name.idxName(), null, idxQryDesc);
-
         return idx;
     }
 
@@ -215,7 +220,7 @@ public class IndexQueryProcessor {
      * @return Index found by list of criteria fields.
      * @throws IgniteCheckedException if suitable index not found.
      */
-    private Index indexByCriteria(
+    private SortedSegmentedIndex indexByCriteria(
         GridCacheContext<?, ?> cctx,
         final Map<String, String> criteriaFlds,
         String tableName,
@@ -224,24 +229,40 @@ public class IndexQueryProcessor {
         Collection<Index> idxs = idxProc.indexes(cctx);
 
         for (Index idx: idxs) {
-            IndexDefinition idxDef = idxProc.indexDefinition(idx.id());
-
-            if (!tableName.equals(idxDef.idxName().tableName()))
-                continue;
+            SortedSegmentedIndex sortedIdx = assertSortedIndex(idx, idxQryDesc);
 
-            if (checkIndex(idxDef, criteriaFlds))
-                return idx;
+            if (checkIndex(sortedIdx, tableName, criteriaFlds))
+                return sortedIdx;
         }
 
         throw failIndexQuery("No index found for criteria", null, idxQryDesc);
     }
 
+    /** Assert if specified index is not an instance of {@link SortedSegmentedIndex}. */
+    private SortedSegmentedIndex assertSortedIndex(Index idx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        if (idx == null)
+            return null;
+
+        if (!(idx instanceof SortedSegmentedIndex))
+            throw failIndexQuery("IndexQuery is not supported for index: " + idx.name(), null, idxQryDesc);
+
+        return (SortedSegmentedIndex)idx;
+    }
+
     /**
-     * Checks that specified index matches index query criteria.
+     * Checks that specified sorted index matches index query criteria.
      *
      * Criteria fields have to match to a prefix of the index. Order of fields in criteria doesn't matter.
      */
-    private boolean checkIndex(IndexDefinition idxDef, Map<String, String> criteriaFlds) {
+    private boolean checkIndex(SortedSegmentedIndex idx, String tblName, Map<String, String> criteriaFlds) {
+        IndexDefinition idxDef = idxProc.indexDefinition(idx.id());
+
+        if (!tblName.equals(idxDef.idxName().tableName()))
+            return false;
+
+        if (F.isEmpty(criteriaFlds))
+            return true;
+
         Map<String, String> flds = new HashMap<>(criteriaFlds);
 
         for (String idxFldName: idxDef.indexKeyDefinitions().keySet()) {
@@ -430,40 +451,35 @@ public class IndexQueryProcessor {
     }
 
     /**
-     * Runs an index query.
+     * Prepare index query.
      *
-     * @return Result cursor over index segments.
+     * @return Prepared query for index range.
      */
-    private GridCursor<IndexRow> query(GridCacheContext<?, ?> cctx, Index idx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx)
-        throws IgniteCheckedException {
+    private IndexRangeQuery prepareQuery(SortedSegmentedIndex idx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        SortedIndexDefinition idxDef = (SortedIndexDefinition) idxProc.indexDefinition(idx.id());
 
-        IndexQueryCriterion c = F.isEmpty(idxQryDesc.criteria()) ? null : idxQryDesc.criteria().get(0);
+        // For PK indexes will serialize _KEY column.
+        if (F.isEmpty(idxQryDesc.criteria()))
+            return new IndexRangeQuery(1);
 
-        if (c == null || c instanceof RangeIndexQueryCriterion)
-            return querySortedIndex(cctx, (InlineIndexImpl) idx, idxQryDesc, qryCtx);
+        InlineIndexImpl sortedIdx = (InlineIndexImpl)idx;
 
-        throw new IllegalStateException("Doesn't support index query criteria: " + c.getClass().getName());
+        Map<String, RangeIndexQueryCriterion> merged = mergeIndexQueryCriteria(sortedIdx, idxDef, idxQryDesc);
+
+        return alignCriteriaWithIndex(sortedIdx, merged, idxDef);
     }
 
     /**
-     * Runs an index query for single {@code segment}.
+     * Runs an index query.
      *
-     * @return Result cursor over segment.
+     * @return Result cursor.
      */
-    private GridCursor<IndexRow> querySortedIndex(GridCacheContext<?, ?> cctx, InlineIndexImpl idx, IndexQueryDesc idxQryDesc,
-        IndexQueryContext qryCtx) throws IgniteCheckedException {
-        SortedIndexDefinition idxDef = (SortedIndexDefinition) idxProc.indexDefinition(idx.id());
-
-        IndexRangeQuery qry;
-
-        if (!F.isEmpty(idxQryDesc.criteria())) {
-            Map<String, RangeIndexQueryCriterion> merged = mergeIndexQueryCriteria(idx, idxDef, idxQryDesc);
-
-            qry = alignCriteriaWithIndex(idx, merged, idxDef);
-        }
-        else
-            qry = new IndexRangeQuery(0);
-
+    private GridCursor<IndexRow> querySortedIndex(
+        GridCacheContext<?, ?> cctx,
+        SortedSegmentedIndex idx,
+        IndexQueryContext qryCtx,
+        IndexRangeQuery qry
+    ) throws IgniteCheckedException {
         int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
 
         if (segmentsCnt == 1)
@@ -475,8 +491,7 @@ public class IndexQueryProcessor {
         for (int i = 0; i < segmentsCnt; i++)
             segmentCursors[i] = treeIndexRange(idx, i, qry, qryCtx);
 
-        return new SegmentedIndexCursor(
-            segmentCursors, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+        return new SegmentedIndexCursor(segmentCursors, (SortedIndexDefinition)idxProc.indexDefinition(idx.id()));
     }
 
     /**
@@ -489,10 +504,10 @@ public class IndexQueryProcessor {
      * 2. To apply criteria on non-first index fields. Tree apply boundaries field by field, if first field match
      * a boundary, then second field isn't checked within traversing.
      */
-    private GridCursor<IndexRow> treeIndexRange(InlineIndexImpl idx, int segment, IndexRangeQuery qry, IndexQueryContext qryCtx)
+    private GridCursor<IndexRow> treeIndexRange(SortedSegmentedIndex idx, int segment, IndexRangeQuery qry, IndexQueryContext qryCtx)
         throws IgniteCheckedException {
 
-        InlineIndexRowHandler hnd = idx.segment(segment).rowHandler();
+        LinkedHashMap<String, IndexKeyDefinition> idxDef = idxProc.indexDefinition(idx.id()).indexKeyDefinitions();
 
         // Step 1. Traverse index.
         GridCursor<IndexRow> findRes = idx.find(qry.lower, qry.upper, segment, qryCtx);
@@ -534,7 +549,7 @@ public class IndexQueryProcessor {
                 for (int i = 0; i < criteriaKeysCnt; i++) {
                     RangeIndexQueryCriterion c = qry.criteria[i];
 
-                    boolean descOrder = hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC;
+                    boolean descOrder = idxDef.get(c.field()).order().sortOrder() == DESC;
 
                     if (low != null && low.key(i) != null) {
                         int cmp = rowCmp.compareRow(row, low, i);
@@ -576,7 +591,7 @@ public class IndexQueryProcessor {
         return key;
     }
 
-    /** Single cursor over multiple segments. Next value is choose with the index row comparator. */
+    /** Single cursor over multiple segments. The next value is chosen with the index row comparator. */
     private static class SegmentedIndexCursor implements GridCursor<IndexRow> {
         /** Cursors over segments. */
         private final PriorityQueue<GridCursor<IndexRow>> cursors;
@@ -588,14 +603,30 @@ public class IndexQueryProcessor {
         private IndexRow head;
 
         /** */
-        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) throws IgniteCheckedException {
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, SortedIndexDefinition idxDef) throws IgniteCheckedException {
             cursorComp = new Comparator<GridCursor<IndexRow>>() {
                 @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
                     try {
-                        return rowCmp.compareRow(o1.get(), o2.get(), 0);
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IgniteException(e);
+                        int keysLen = o1.get().keys().length;
+
+                        Iterator<IndexKeyDefinition> it = idxDef.indexKeyDefinitions().values().iterator();
+
+                        for (int i = 0; i < keysLen; i++) {
+                            int cmp = idxDef.rowComparator().compareRow(o1.get(), o2.get(), i);
+
+                            IndexKeyDefinition def = it.next();
+
+                            if (cmp != 0) {
+                                boolean desc = def.order().sortOrder() == SortOrder.DESC;
+
+                                return desc ? -cmp : cmp;
+                            }
+                        }
+
+                        return 0;
+
+                    } catch (IgniteCheckedException e) {
+                        throw new IgniteException("Failed to sort remote index rows", e);
                     }
                 }
             };
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryResult.java
similarity index 52%
copy from modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryResult.java
index 28bc04d..9b9d2df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryResult.java
@@ -15,34 +15,32 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.cache.query.index.sorted.keys;
+package org.apache.ignite.internal.cache.query.index;
 
-import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.util.lang.GridCloseableIterator;
+import org.apache.ignite.lang.IgniteBiTuple;
 
-/** */
-public class BooleanIndexKey implements IndexKey {
-    /** */
-    private final boolean key;
+/** Represents result of local execution of IndexQuery. */
+public class IndexQueryResult<K, V> {
+    /** Result data iterator. */
+    private final GridCloseableIterator<IgniteBiTuple<K, V>> iter;
 
-    /** */
-    public BooleanIndexKey(boolean key) {
-        this.key = key;
-    }
+    /** Result metadata. */
+    private final IndexQueryResultMeta metadata;
 
-    /** {@inheritDoc} */
-    @Override public Object key() {
-        return key;
+    /** */
+    public IndexQueryResult(IndexQueryResultMeta metadata, GridCloseableIterator<IgniteBiTuple<K, V>> iter) {
+        this.iter = iter;
+        this.metadata = metadata;
     }
 
-    /** {@inheritDoc} */
-    @Override public int type() {
-        return IndexKeyTypes.BOOLEAN;
+    /** */
+    public GridCloseableIterator<IgniteBiTuple<K, V>> iter() {
+        return iter;
     }
 
-    /** {@inheritDoc} */
-    @Override public int compare(IndexKey o) {
-        boolean okey = (boolean) o.key();
-
-        return Boolean.compare(key, okey);
+    /** */
+    public IndexQueryResultMeta metadata() {
+        return metadata;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryResultMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryResultMeta.java
new file mode 100644
index 0000000..96e5ff7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryResultMeta.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.MetaPageInfo;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Metadata for IndexQuery response. This information is required to be sent to a node that initiated a query.
+ * Thick client nodes may have irrelevant information about index structure, {@link MetaPageInfo}.
+ */
+public class IndexQueryResultMeta implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index key settings. */
+    private IndexKeyTypeSettings keyTypeSettings;
+
+    /** Index key definitions. */
+    private LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+    /** */
+    public IndexQueryResultMeta() {
+        // No-op.
+    }
+
+    /** */
+    public IndexQueryResultMeta(SortedIndexDefinition def, int critSize) {
+        keyTypeSettings = def.keyTypeSettings();
+
+        keyDefs = new LinkedHashMap<>();
+
+        Iterator<Map.Entry<String, IndexKeyDefinition>> keys = def.indexKeyDefinitions().entrySet().iterator();
+
+        for (int i = 0; i < critSize; i++) {
+            Map.Entry<String, IndexKeyDefinition> key = keys.next();
+
+            keyDefs.put(key.getKey(), key.getValue());
+        }
+    }
+
+    /** */
+    public IndexKeyTypeSettings keyTypeSettings() {
+        return keyTypeSettings;
+    }
+
+    /** */
+    public LinkedHashMap<String, IndexKeyDefinition> keyDefinitions() {
+        return keyDefs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(keyTypeSettings);
+
+        U.writeMap(out, keyDefs);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        keyTypeSettings = (IndexKeyTypeSettings)in.readObject();
+
+        keyDefs = U.readLinkedMap(in);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java
index c0e51db..e1699de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java
@@ -17,22 +17,36 @@
 
 package org.apache.ignite.internal.cache.query.index.sorted;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.internal.cache.query.index.Order;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  * Defines a signle index key.
  */
-public class IndexKeyDefinition {
+public class IndexKeyDefinition implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Index key type. {@link IndexKeyTypes}. */
-    private final int idxType;
+    private int idxType;
 
     /** Order. */
-    private final Order order;
+    private Order order;
 
     /** Precision for variable length key types. */
-    private final int precision;
+    private int precision;
+
+    /** */
+    public IndexKeyDefinition() {
+        // No-op.
+    }
 
     /** */
     public IndexKeyDefinition(int idxType, Order order, long precision) {
@@ -70,4 +84,17 @@ public class IndexKeyDefinition {
 
         return idxType == key.type();
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        // Send only required info for using in MergeSort algorithm.
+        out.writeInt(idxType);
+        U.writeEnum(out, order.sortOrder());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        idxType = in.readInt();
+        order = new Order(U.readEnum(in, SortOrder.class), null);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java
index 09e2092..095974a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java
@@ -17,10 +17,18 @@
 
 package org.apache.ignite.internal.cache.query.index.sorted;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
 /**
  * List of settings that affects key types of index keys.
  */
-public class IndexKeyTypeSettings {
+public class IndexKeyTypeSettings implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Whether inlining POJO keys as hash is supported. */
     private boolean inlineObjHash = true;
 
@@ -78,4 +86,20 @@ public class IndexKeyTypeSettings {
 
         return this;
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeBoolean(inlineObjHash);
+        out.writeBoolean(inlineObjSupported);
+        out.writeBoolean(strOptimizedCompare);
+        out.writeBoolean(binaryUnsigned);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        inlineObjHash = in.readBoolean();
+        inlineObjSupported = in.readBoolean();
+        strOptimizedCompare = in.readBoolean();
+        binaryUnsigned = in.readBoolean();
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java
index 916fe25..f42a9ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java
@@ -33,14 +33,6 @@ import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.N
  * 2. Comparison of different types is not supported.
  */
 public class IndexRowCompartorImpl implements IndexRowComparator {
-    /** Key type settings for this index. */
-    protected final IndexKeyTypeSettings keyTypeSettings;
-
-    /** */
-    public IndexRowCompartorImpl(IndexKeyTypeSettings keyTypeSettings) {
-        this.keyTypeSettings = keyTypeSettings;
-    }
-
     /** {@inheritDoc} */
     @Override public int compareKey(long pageAddr, int off, int maxSize, IndexKey key, int curType) {
         if (curType == IndexKeyTypes.UNKNOWN)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
index 28bc04d..75fcbd3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
@@ -41,8 +41,6 @@ public class BooleanIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        boolean okey = (boolean) o.key();
-
-        return Boolean.compare(key, okey);
+        return Boolean.compare(key, ((BooleanIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java
index cd1842b..aed895f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java
@@ -41,8 +41,6 @@ public class ByteIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        byte okey = (byte) o.key();
-
-        return Byte.compare(key, okey);
+        return Byte.compare(key, ((ByteIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java
index 8ebf857..ed9a2db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java
@@ -41,8 +41,6 @@ public class BytesIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        byte[] okey = (byte[]) o.key();
-
-        return BytesCompareUtils.compareNotNullUnsigned(key, okey);
+        return BytesCompareUtils.compareNotNullUnsigned(key, ((BytesIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java
index 9967189d..69d2a9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java
@@ -41,8 +41,6 @@ public class DoubleIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        double okey = (double) o.key();
-
-        return Double.compare(key, okey);
+        return Double.compare(key, ((DoubleIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java
index 32986a6..4384e98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java
@@ -41,8 +41,6 @@ public class FloatIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        float okey = (float) o.key();
-
-        return Float.compare(key, okey);
+        return Float.compare(key, ((FloatIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/IntegerIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/IntegerIndexKey.java
index 9f7acf9..0164119 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/IntegerIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/IntegerIndexKey.java
@@ -41,8 +41,6 @@ public class IntegerIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        int okey = (int) o.key();
-
-        return Integer.compare(key, okey);
+        return Integer.compare(key, ((IntegerIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/LongIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/LongIndexKey.java
index 1a5a4e9..59c13d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/LongIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/LongIndexKey.java
@@ -41,8 +41,6 @@ public class LongIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        long okey = (long) o.key();
-
-        return Long.compare(key, okey);
+        return Long.compare(key, ((LongIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ShortIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ShortIndexKey.java
index afd6b09..f688053 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ShortIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ShortIndexKey.java
@@ -41,9 +41,7 @@ public class ShortIndexKey implements IndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        short okey = (short) o.key();
-
         // Keep old logic there.
-        return Integer.compare(key, okey);
+        return Integer.compare(key, ((ShortIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/SignedBytesIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/SignedBytesIndexKey.java
index 56d4565..0e52409 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/SignedBytesIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/SignedBytesIndexKey.java
@@ -26,8 +26,6 @@ public class SignedBytesIndexKey extends BytesIndexKey {
 
     /** {@inheritDoc} */
     @Override public int compare(IndexKey o) {
-        byte[] okey = (byte[]) o.key();
-
-        return BytesCompareUtils.compareNotNullSigned(key, okey);
+        return BytesCompareUtils.compareNotNullSigned(key, ((BytesIndexKey)o).key);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
index a730347..7235d3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
@@ -59,12 +59,12 @@ public class GridCacheDistributedFieldsQueryFuture
      * @param err Error.
      * @param finished Finished or not.
      */
-    public void onPage(@Nullable UUID nodeId, @Nullable List<GridQueryFieldMetadata> metaData,
+    public void onFieldsPage(@Nullable UUID nodeId, @Nullable List<GridQueryFieldMetadata> metaData,
         @Nullable Collection<Map<String, Object>> data, @Nullable Throwable err, boolean finished) {
         if (!metaFut.isDone() && metaData != null)
             metaFut.onDone(metaData);
 
-        onPage(nodeId, data, err, finished);
+        onPage(nodeId, null, data, err, finished);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
index 1efc895..b782a96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
@@ -29,14 +30,17 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.IndexQueryReducer;
 import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.TextQueryReducer;
 import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedCacheQueryReducer;
 import org.apache.ignite.internal.util.lang.GridPlainCallable;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.INDEX;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
 
 /**
@@ -61,6 +65,9 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
     /** Set of nodes that deliver their first page. */
     private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
+    /** Metadata for IndexQuery. */
+    private final CompletableFuture<IndexQueryResultMeta> idxQryMetaFut;
+
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
@@ -89,9 +96,16 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
 
         Map<UUID, NodePageStream<R>> streamsMap = Collections.unmodifiableMap(streams);
 
-        reducer = qry.query().type() == TEXT ?
-            new MergeSortCacheQueryReducer<>(streamsMap)
-            : new UnsortedCacheQueryReducer<>(streamsMap);
+        if (qry.query().type() == INDEX) {
+            idxQryMetaFut = new CompletableFuture<>();
+
+            reducer = new IndexQueryReducer<>(qry.query().idxQryDesc().valType(), streamsMap, cctx, idxQryMetaFut);
+        }
+        else {
+            idxQryMetaFut = null;
+
+            reducer = qry.query().type() == TEXT ? new TextQueryReducer<>(streamsMap) : new UnsortedCacheQueryReducer<>(streamsMap);
+        }
     }
 
     /** {@inheritDoc} */
@@ -150,6 +164,12 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
     }
 
     /** {@inheritDoc} */
+    @Override protected void onMeta(IndexQueryResultMeta metaData) {
+        if (metaData != null)
+            idxQryMetaFut.complete(metaData);
+    }
+
+    /** {@inheritDoc} */
     @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
         U.await(firstPageLatch);
 
@@ -270,6 +290,9 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
         if (onDone(err)) {
             streams.values().forEach(s -> s.cancel(err));
 
+            if (idxQryMetaFut != null && !idxQryMetaFut.isDone())
+                idxQryMetaFut.completeExceptionally(err);
+
             firstPageLatch.countDown();
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index 6fb0907..25a145f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -32,6 +32,7 @@ import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
@@ -58,6 +59,7 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.INDEX;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SCAN;
 
 /**
@@ -139,7 +141,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
         for (Map.Entry<Long, GridCacheDistributedQueryFuture<?, ?, ?>> e : futs.entrySet()) {
             GridCacheDistributedQueryFuture<?, ?, ?> fut = e.getValue();
 
-            fut.onPage(null, null, err, true);
+            fut.onPage(null, null, null, err, true);
 
             futs.remove(e.getKey(), fut);
         }
@@ -377,14 +379,14 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
         if (fut != null)
             if (res.fields())
-                ((GridCacheDistributedFieldsQueryFuture)fut).onPage(
+                ((GridCacheDistributedFieldsQueryFuture)fut).onFieldsPage(
                     sndId,
                     res.metadata(),
                     (Collection<Map<String, Object>>)((Collection)res.data()),
                     res.error(),
                     res.isFinished());
             else
-                fut.onPage(sndId, res.data(), res.error(), res.isFinished());
+                fut.onPage(sndId, res.idxQryMetadata(), res.data(), res.error(), res.isFinished());
         else if (!cancelled.contains(res.requestId()))
             U.warn(log, "Received response for finished or unknown query [rmtNodeId=" + sndId +
                 ", res=" + res + ']');
@@ -411,8 +413,14 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPageReady(boolean loc, GridCacheQueryInfo qryInfo,
-        Collection<?> data, boolean finished, Throwable e) {
+    @Override protected boolean onPageReady(
+        boolean loc,
+        GridCacheQueryInfo qryInfo,
+        IndexQueryResultMeta idxQryMetadata,
+        Collection<?> data,
+        boolean finished,
+        Throwable e
+    ) {
         GridCacheLocalQueryFuture<?, ?, ?> fut = qryInfo.localQueryFuture();
 
         if (loc)
@@ -420,7 +428,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
         if (e != null) {
             if (loc)
-                fut.onPage(null, null, e, true);
+                fut.onPage(null, null, null, e, true);
             else
                 sendQueryResponse(qryInfo.senderId(),
                     new GridCacheQueryResponse(cctx.cacheId(), qryInfo.requestId(), e, cctx.deploymentEnabled()),
@@ -430,13 +438,15 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
         }
 
         if (loc)
-            fut.onPage(null, data, null, finished);
+            fut.onPage(null, null, data, null, finished);
         else {
             GridCacheQueryResponse res = new GridCacheQueryResponse(cctx.cacheId(), qryInfo.requestId(),
-                /*finished*/false, /*fields*/false, cctx.deploymentEnabled());
+                finished, /*fields*/false, cctx.deploymentEnabled());
+
+            if (qryInfo.query().type() == INDEX)
+                res.idxQryMetadata((IndexQueryResultMeta)idxQryMetadata);
 
             res.data(data);
-            res.finished(finished);
 
             if (!sendQueryResponse(qryInfo.senderId(), res, qryInfo.query().timeout()))
                 return false;
@@ -471,7 +481,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
         if (loc) {
             GridCacheLocalFieldsQueryFuture fut = (GridCacheLocalFieldsQueryFuture)qryInfo.localQueryFuture();
 
-            fut.onPage(null, metadata, data, null, finished);
+            fut.onFieldsPage(null, metadata, data, null, finished);
         }
         else {
             GridCacheQueryResponse res = new GridCacheQueryResponse(cctx.cacheId(), qryInfo.requestId(),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
index 20e4066..55057b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
@@ -58,9 +58,9 @@ public class GridCacheLocalFieldsQueryFuture
      * @param err Error.
      * @param finished Finished or not.
      */
-    public void onPage(@Nullable UUID nodeId, @Nullable List<GridQueryFieldMetadata> metaData,
+    public void onFieldsPage(@Nullable UUID nodeId, @Nullable List<GridQueryFieldMetadata> metaData,
         @Nullable Collection<?> data, @Nullable Throwable err, boolean finished) {
-        onPage(nodeId, data, err, finished);
+        onPage(nodeId, null, data, err, finished);
 
         if (!metaFut.isDone())
             metaFut.onDone(metaData);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
index 4d1e20e..1e1f2f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.jetbrains.annotations.Nullable;
@@ -37,6 +38,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
     @Override protected boolean onPageReady(
         boolean loc,
         GridCacheQueryInfo qryInfo,
+        IndexQueryResultMeta metadata,
         Collection<?> data,
         boolean finished, Throwable e) {
         GridCacheQueryFutureAdapter fut = qryInfo.localQueryFuture();
@@ -44,9 +46,9 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
         assert fut != null;
 
         if (e != null)
-            fut.onPage(null, null, e, true);
+            fut.onPage(null, null, null, e, true);
         else
-            fut.onPage(null, data, null, finished);
+            fut.onPage(null, metadata, data, null, finished);
 
         return true;
     }
@@ -68,7 +70,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
         if (e != null)
             fut.onPage(null, null, null, e, true);
         else
-            fut.onPage(null, metaData, data, null, finished);
+            fut.onFieldsPage(null, metaData, data, null, finished);
 
         return true;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index 691ea1c..b40c134 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
 import org.apache.ignite.internal.processors.cache.CacheObjectUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.query.reducer.CacheQueryReducer;
@@ -149,7 +150,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
             R next = null;
 
             if (reducer.hasNextX()) {
-                next = unmaskNull(reducer.nextX());
+                next = unmaskNull(unwrapIfNeeded(reducer.nextX()));
 
                 if (!limitDisabled) {
                     cnt++;
@@ -207,11 +208,18 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
      * Entrypoint for handling query result page from remote node.
      *
      * @param nodeId Sender node.
+     * @param metadata Query response metadata.
      * @param data Page data.
      * @param err Error (if was).
      * @param lastPage Whether it is the last page for sender node.
      */
-    public void onPage(@Nullable UUID nodeId, @Nullable Collection<?> data, @Nullable Throwable err, boolean lastPage) {
+    public void onPage(
+        @Nullable UUID nodeId,
+        @Nullable IndexQueryResultMeta metadata,
+        @Nullable Collection<?> data,
+        @Nullable Throwable err,
+        boolean lastPage
+    ) {
         if (isCancelled())
             return;
 
@@ -260,8 +268,13 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
 
                     data = unwrapped;
 
-                } else
+                } else if (qry.query().type() != GridCacheQueryType.INDEX) {
+                    // For IndexQuery BinaryObjects are used for sorting algorithm.
                     data = cctx.unwrapBinariesIfNeeded((Collection<Object>)data, qry.query().keepBinary());
+                }
+
+                if (query().query().type() == GridCacheQueryType.INDEX)
+                    onMeta(metadata);
 
                 onPage(nodeId, (Collection<R>) data, lastPage);
 
@@ -283,6 +296,11 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     /** Handles new data page from query node. */
     protected abstract void onPage(UUID nodeId, Collection<R> data, boolean lastPage);
 
+    /** Handles query meta data from query node. */
+    protected void onMeta(IndexQueryResultMeta meta) {
+        // No-op.
+    }
+
     /** {@inheritDoc} */
     @Override public boolean onDone(Collection<R> res, Throwable err) {
         boolean done = super.onDone(res, err);
@@ -328,6 +346,14 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
         return obj != NULL ? obj : null;
     }
 
+    /** */
+    private R unwrapIfNeeded(R obj) {
+        if (qry.query().type() == GridCacheQueryType.INDEX)
+            return (R)cctx.unwrapBinaryIfNeeded(obj, qry.query().keepBinary(), false, null);
+
+        return obj;
+    }
+
     /**
      * Clears future.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index d02316b..ced4e4a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -38,6 +38,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.BiFunction;
@@ -61,6 +62,8 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResult;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.metric.IoStatisticsQueryHelper;
@@ -626,8 +629,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     break;
 
                 case INDEX:
-                    iter = qryProc.queryIndex(cacheName, qry.queryClassName(), qry.idxQryDesc(), qry.scanFilter(),
-                        filter(qry), qry.keepBinary());
+                    IndexQueryResult<K, V> idxQryRes = qryProc.queryIndex(cacheName, qry.queryClassName(), qry.idxQryDesc(),
+                        qry.scanFilter(), filter(qry), qry.keepBinary());
+
+                    iter = idxQryRes.iter();
+                    res.metadata(idxQryRes.metadata());
 
                     break;
 
@@ -988,7 +994,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                     // Query is cancelled.
                     if (row == null) {
-                        onPageReady(qryInfo.local(), qryInfo, null, true, null);
+                        onPageReady(qryInfo.local(), qryInfo, null, null, true, null);
 
                         break;
                     }
@@ -1191,7 +1197,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                     // Query is cancelled.
                     if (row0 == null) {
-                        onPageReady(loc, qryInfo, null, true, null);
+                        onPageReady(loc, qryInfo, null, null, true, null);
 
                         break;
                     }
@@ -1295,7 +1301,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                             // Reduce.
                             if (!rdc.collect(entry) || !iter.hasNext()) {
-                                onPageReady(loc, qryInfo, Collections.singletonList(rdc.reduce()), true, null);
+                                onPageReady(loc, qryInfo, null, Collections.singletonList(rdc.reduce()), true, null);
 
                                 pageSent = true;
 
@@ -1317,10 +1323,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         if (++cnt == pageSize || !iter.hasNext()) {
                             boolean finished = !iter.hasNext();
 
-                            onPageReady(loc, qryInfo, data, finished, null);
+                            onPageReady(loc, qryInfo, res.metadata(), data, finished, null);
 
                             pageSent = true;
 
+                            res.onPageSend();
+
                             if (!finished)
                                 rmvIter = false;
 
@@ -1337,9 +1345,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 if (!pageSent) {
                     if (rdc == null)
-                        onPageReady(loc, qryInfo, data, true, null);
+                        onPageReady(loc, qryInfo, res.metadata(), data, true, null);
                     else
-                        onPageReady(loc, qryInfo, Collections.singletonList(rdc.reduce()), true, null);
+                        onPageReady(loc, qryInfo, res.metadata(), Collections.singletonList(rdc.reduce()), true, null);
+
+                    res.onPageSend();
                 }
             }
             catch (Throwable e) {
@@ -1358,7 +1368,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 if (!X.hasCause(e, GridDhtUnreservedPartitionException.class))
                     U.error(log, "Failed to run query [qry=" + qryInfo + ", node=" + cctx.nodeId() + "]", e);
 
-                onPageReady(loc, qryInfo, null, true, e);
+                onPageReady(loc, qryInfo, null, null, true, e);
 
                 if (e instanceof Error)
                     throw (Error)e;
@@ -1709,12 +1719,13 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      *
      * @param loc Local query or not.
      * @param qryInfo Query info.
+     * @param metaData Meta data.
      * @param data Result data.
      * @param finished Last page or not.
      * @param e Exception in case of error.
      * @return {@code true} if page was processed right.
      */
-    protected abstract boolean onPageReady(boolean loc, GridCacheQueryInfo qryInfo,
+    protected abstract boolean onPageReady(boolean loc, GridCacheQueryInfo qryInfo, @Nullable IndexQueryResultMeta metaData,
         @Nullable Collection<?> data, boolean finished, @Nullable Throwable e);
 
     /**
@@ -2395,11 +2406,14 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      */
     public static class QueryResult<K, V> extends CachedResult<IgniteBiTuple<K, V>> {
         /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
         private final GridCacheQueryType type;
 
+        /** Future of query result metadata. Completed when query actually started. */
+        private final CompletableFuture<IndexQueryResultMeta> metadata;
+
+        /** Flag shows whether first result page was delivered to user. */
+        private volatile boolean sentFirst;
+
         /**
          * @param type Query type.
          * @param rcpt ID of the recipient.
@@ -2408,6 +2422,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             super(rcpt);
 
             this.type = type;
+
+            metadata = type == INDEX ? new CompletableFuture<>() : null;
         }
 
         /**
@@ -2416,6 +2432,37 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         public GridCacheQueryType type() {
             return type;
         }
+
+        /** */
+        public IndexQueryResultMeta metadata() {
+            if (sentFirst || metadata == null)
+                return null;
+
+            assert metadata.isDone() : "QueryResult metadata isn't completed yet.";
+
+            return metadata.getNow(null);
+        }
+
+        /** */
+        public void metadata(IndexQueryResultMeta metadata) {
+            if (this.metadata != null)
+                this.metadata.complete(metadata);
+        }
+
+        /** Callback to invoke, when next data page was delivered to user. */
+        public void onPageSend() {
+            sentFirst = true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onDone(@Nullable IgniteSpiCloseableIterator<IgniteBiTuple<K, V>> res, @Nullable Throwable err) {
+            boolean done = super.onDone(res, err);
+
+            if (done && err != null && metadata != null)
+                metadata.completeExceptionally(err);
+
+            return done;
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
index 8e05c02..99de925 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
@@ -44,7 +45,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Query request.
+ * Page of cache query response.
  */
 public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCacheDeployable {
     /** */
@@ -76,6 +77,13 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
     private List<GridQueryFieldMetadata> metadata;
 
     /** */
+    @GridDirectTransient
+    private IndexQueryResultMeta idxQryMetadata;
+
+    /** */
+    private byte[] idxQryMetadataBytes;
+
+    /** */
     @GridDirectCollection(byte[].class)
     private Collection<byte[]> dataBytes;
 
@@ -133,6 +141,9 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
         if (metaDataBytes == null && metadata != null)
             metaDataBytes = marshalCollection(metadata, cctx);
 
+        if (idxQryMetadataBytes == null && idxQryMetadata != null)
+            idxQryMetadataBytes = U.marshal(ctx, idxQryMetadata);
+
         if (dataBytes == null && data != null)
             dataBytes = marshalCollection(data, cctx);
 
@@ -158,6 +169,9 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
         if (metadata == null)
             metadata = unmarshalCollection(metaDataBytes, ctx, ldr);
 
+        if (idxQryMetadataBytes != null && idxQryMetadata == null)
+            idxQryMetadata = U.unmarshal(ctx, idxQryMetadataBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+
         if (data == null)
             data = unmarshalCollection0(dataBytes, ctx, ldr);
     }
@@ -218,6 +232,13 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
     }
 
     /**
+     * @return IndexQuery metadata.
+     */
+    public IndexQueryResultMeta idxQryMetadata() {
+        return idxQryMetadata;
+    }
+
+    /**
      * @param metadata Metadata.
      */
     public void metadata(@Nullable List<GridQueryFieldMetadata> metadata) {
@@ -225,6 +246,13 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
     }
 
     /**
+     * @param idxQryMetadata IndexQuery metadata.
+     */
+    public void idxQryMetadata(IndexQueryResultMeta idxQryMetadata) {
+        this.idxQryMetadata = idxQryMetadata;
+    }
+
+    /**
      * @return Query data.
      */
     public Collection<Object> data() {
@@ -322,6 +350,11 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 writer.incrementState();
 
+            case 10:
+                if (!writer.writeByteArray("idxQryMetadataBytes", idxQryMetadataBytes))
+                    return false;
+
+                writer.incrementState();
         }
 
         return true;
@@ -386,6 +419,13 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
 
                 reader.incrementState();
 
+            case 10:
+                idxQryMetadataBytes = reader.readByteArray("idxQryMetadataBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
         }
 
         return reader.afterMessageRead(GridCacheQueryResponse.class);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/CacheQueryReducer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/CacheQueryReducer.java
index abbe949..b827c2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/CacheQueryReducer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/CacheQueryReducer.java
@@ -47,11 +47,12 @@ public abstract class CacheQueryReducer<T> extends GridIteratorAdapter<T> {
     }
 
     /**
-     * @return Page with query results data from specified stream.
+     * @return Object that completed the specified future.
+     * @throws IgniteCheckedException for all failures.
      */
-    public static <T> NodePage<T> get(CompletableFuture<?> pageFut) throws IgniteCheckedException {
+    public static <T> T get(CompletableFuture<?> fut) throws IgniteCheckedException {
         try {
-            return (NodePage<T>) pageFut.get();
+            return (T) fut.get();
         }
         catch (InterruptedException e) {
             Thread.currentThread().interrupt();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/IndexQueryReducer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/IndexQueryReducer.java
new file mode 100644
index 0000000..75ac0c4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/IndexQueryReducer.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.reducer;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowComparator;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCompartorImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Reducer for {@code IndexQuery} results.
+ */
+public class IndexQueryReducer<R> extends MergeSortCacheQueryReducer<R> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Future that will be completed with first page response. */
+    private final CompletableFuture<IndexQueryResultMeta> metaFut;
+
+    /** */
+    private final String valType;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    public IndexQueryReducer(
+        final String valType,
+        final Map<UUID, NodePageStream<R>> pageStreams,
+        final GridCacheContext<?, ?> cctx,
+        final CompletableFuture<IndexQueryResultMeta> meta
+    ) {
+        super(pageStreams);
+
+        this.valType = valType;
+        this.metaFut = meta;
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CompletableFuture<Comparator<NodePage<R>>> pageComparator() {
+        return metaFut.thenApply(m -> {
+            LinkedHashMap<String, IndexKeyDefinition> keyDefs = m.keyDefinitions();
+
+            GridQueryTypeDescriptor typeDesc = cctx.kernalContext().query().typeDescriptor(cctx.name(), QueryUtils.typeName(valType));
+
+            return new IndexedNodePageComparator(m, typeDesc, keyDefs);
+        });
+    }
+
+    /** Comparing rows by indexed keys. */
+    private class IndexedNodePageComparator implements Comparator<NodePage<R>>, Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Index key defintiions in case of IndexQuery. */
+        private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+        /** Description of value type for IndexQuery. */
+        private final GridQueryTypeDescriptor typeDesc;
+
+        /** IndexQuery meta. */
+        private final IndexQueryResultMeta meta;
+
+        /** Every node will return the same key types for the same index, then it's possible to use simple comparator. */
+        private final IndexRowComparator idxRowComp = new IndexRowCompartorImpl();
+
+        /** */
+        IndexedNodePageComparator(
+            IndexQueryResultMeta meta,
+            GridQueryTypeDescriptor typeDesc,
+            LinkedHashMap<String, IndexKeyDefinition> keyDefs
+        ) {
+            this.meta = meta;
+            this.typeDesc = typeDesc;
+            this.keyDefs = keyDefs;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compare(NodePage<R> o1, NodePage<R> o2) {
+            IgniteBiTuple<?, ?> e1 = (IgniteBiTuple<?, ?>)o1.head();
+            IgniteBiTuple<?, ?> e2 = (IgniteBiTuple<?, ?>)o2.head();
+
+            Iterator<Map.Entry<String, IndexKeyDefinition>> defs = keyDefs.entrySet().iterator();
+
+            try {
+                while (defs.hasNext()) {
+                    Map.Entry<String, IndexKeyDefinition> d = defs.next();
+
+                    IndexKey k1 = key(d.getKey(), d.getValue().idxType(), e1);
+                    IndexKey k2 = key(d.getKey(), d.getValue().idxType(), e2);
+
+                    int cmp = idxRowComp.compareKey(k1, k2);
+
+                    if (cmp != 0)
+                        return d.getValue().order().sortOrder() == DESC ? -cmp : cmp;
+                }
+
+                return 0;
+
+            } catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to sort remote index rows", e);
+            }
+        }
+
+        /** */
+        private IndexKey key(String key, int type, IgniteBiTuple<?, ?> entry) throws IgniteCheckedException {
+            GridQueryProperty prop = typeDesc.property(key);
+
+            // PrimaryKey field.
+            Object o = prop == null ? entry.getKey() : prop.value(entry.getKey(), entry.getValue());
+
+            return IndexKeyFactory.wrap(o, type, cctx.cacheObjectContext(), meta.keyTypeSettings());
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortCacheQueryReducer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortCacheQueryReducer.java
index d2b3a3b..3024a07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortCacheQueryReducer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortCacheQueryReducer.java
@@ -22,14 +22,14 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.PriorityQueue;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.cache.query.ScoredCacheEntry;
 
 /**
  * Reducer of cache query results that sort result through all nodes. Note that it's assumed that every node
  * returns pre-sorted collection of data.
  */
-public class MergeSortCacheQueryReducer<R> extends CacheQueryReducer<R> {
+abstract class MergeSortCacheQueryReducer<R> extends CacheQueryReducer<R> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -43,19 +43,19 @@ public class MergeSortCacheQueryReducer<R> extends CacheQueryReducer<R> {
     private UUID pendingNodeId;
 
     /** */
-    public MergeSortCacheQueryReducer(final Map<UUID, NodePageStream<R>> pageStreams) {
+    protected MergeSortCacheQueryReducer(final Map<UUID, NodePageStream<R>> pageStreams) {
         super(pageStreams);
     }
 
+    /** @return Comparator for pages from nodes. */
+    protected abstract CompletableFuture<Comparator<NodePage<R>>> pageComparator();
+
     /** {@inheritDoc} */
     @Override public boolean hasNextX() throws IgniteCheckedException {
         // Initial sort.
         if (nodePages == null) {
             // Compares head pages from all nodes to get the lowest value at the moment.
-            Comparator<NodePage<R>> pageCmp = (o1, o2) -> textResultComparator.compare(
-                (ScoredCacheEntry<?, ?>)o1.head(), (ScoredCacheEntry<?, ?>)o2.head());
-
-            nodePages = new PriorityQueue<>(pageStreams.size(), pageCmp);
+            nodePages = new PriorityQueue<>(pageStreams.size(), get(pageComparator()));
 
             for (NodePageStream<R> s : pageStreams.values()) {
                 NodePage<R> p = get(s.headPage());
@@ -76,6 +76,8 @@ public class MergeSortCacheQueryReducer<R> extends CacheQueryReducer<R> {
                 if (p != null && p.hasNext())
                     nodePages.add(p);
             }
+
+            pendingNodeId = null;
         }
 
         return !nodePages.isEmpty();
@@ -97,8 +99,4 @@ public class MergeSortCacheQueryReducer<R> extends CacheQueryReducer<R> {
 
         return o;
     }
-
-    /** Compares rows for {@code TextQuery} results for ordering results in MergeSort reducer. */
-    private static final Comparator<ScoredCacheEntry<?, ?>> textResultComparator = (c1, c2) ->
-        Float.compare(c2.score(), c1.score());
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/TextQueryReducer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/TextQueryReducer.java
new file mode 100644
index 0000000..5ad5a04
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/TextQueryReducer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.reducer;
+
+import java.util.Comparator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.processors.cache.query.ScoredCacheEntry;
+
+/**
+ * Reducer for {@code TextQuery} results.
+ */
+public class TextQueryReducer<R> extends MergeSortCacheQueryReducer<R> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    public TextQueryReducer(final Map<UUID, NodePageStream<R>> pageStreams) {
+        super(pageStreams);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CompletableFuture<Comparator<NodePage<R>>> pageComparator() {
+        CompletableFuture<Comparator<NodePage<R>>> f = new CompletableFuture<>();
+
+        f.complete((o1, o2) -> -Float.compare(
+            ((ScoredCacheEntry<?, ?>)o1.head()).score(), ((ScoredCacheEntry<?, ?>)o2.head()).score()));
+
+        return f;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 654adf6..5ada406 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -63,6 +63,7 @@ import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.cache.query.index.IndexQueryProcessor;
+import org.apache.ignite.internal.cache.query.index.IndexQueryResult;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexQueryContext;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -3374,7 +3375,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Key/value rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryIndex(
+    public <K, V> IndexQueryResult<K, V> queryIndex(
         String cacheName,
         String valCls,
         final IndexQueryDesc idxQryDesc,
@@ -3389,8 +3390,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             final GridCacheContext<K, V> cctx = (GridCacheContext<K, V>) ctx.cache().internalCache(cacheName).context();
 
             return executeQuery(GridCacheQueryType.INDEX, valCls, cctx,
-                new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
-                    @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
+                new IgniteOutClosureX<IndexQueryResult<K, V>>() {
+                    @Override public IndexQueryResult<K, V> applyx() throws IgniteCheckedException {
                         IndexQueryContext qryCtx = new IndexQueryContext(filters, null);
 
                         return idxQryPrc.queryLocal(cctx, idxQryDesc, filter, qryCtx, keepBinary);
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/H2RowComparator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/H2RowComparator.java
index 5530b85..d9722fd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/H2RowComparator.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/H2RowComparator.java
@@ -49,11 +49,14 @@ public class H2RowComparator extends IndexRowCompartorImpl {
     /** Ignite H2 session. */
     private final SessionInterface ses;
 
+    /** Key type settings for this index. */
+    private final IndexKeyTypeSettings keyTypeSettings;
+
     /** */
     public H2RowComparator(GridH2Table table, IndexKeyTypeSettings keyTypeSettings) {
-        super(keyTypeSettings);
-
         this.table = table;
+        this.keyTypeSettings = keyTypeSettings;
+
         coctx = table.rowDescriptor().context().cacheObjectContext();
         ses = table.rowDescriptor().indexing().connections().jdbcConnection().getSession();
     }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAliasTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAliasTest.java
index 9b49518..6e49453 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAliasTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAliasTest.java
@@ -23,9 +23,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.LongStream;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -34,7 +31,9 @@ import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -120,13 +119,13 @@ public class IndexQueryAliasTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("asId", pivot));
 
-        check(cache.query(qry), 0, pivot);
+        check(cache.query(qry), 0, pivot, false);
 
         // Lt, desc index.
         IndexQuery<Long, Person> descQry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
             .setCriteria(lt("asDescId", pivot));
 
-        check(cache.query(descQry), 0, pivot);
+        check(cache.query(descQry), 0, pivot, true);
     }
 
     /** */
@@ -140,7 +139,7 @@ public class IndexQueryAliasTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, idIdx)
             .setCriteria(lt("ASID", pivot));
 
-        check(cache.query(qry), 0, pivot);
+        check(cache.query(qry), 0, pivot, false);
 
         String idDescIdx = qryDescIdx != null ? qryDescIdx.toLowerCase() : null;
 
@@ -148,35 +147,33 @@ public class IndexQueryAliasTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> descQry = new IndexQuery<Long, Person>(Person.class, idDescIdx)
             .setCriteria(lt("ASDESCID", pivot));
 
-        check(cache.query(descQry), 0, pivot);
+        check(cache.query(descQry), 0, pivot, true);
     }
 
     /**
      * @param left First cache key, inclusive.
      * @param right Last cache key, exclusive.
      */
-    private void check(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {
+    private void check(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right, boolean desc) {
         List<Cache.Entry<Long, Person>> all = cursor.getAll();
 
         assertEquals(right - left, all.size());
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
-
         for (int i = 0; i < all.size(); i++) {
-            Cache.Entry<Long, Person> entry = all.get(i);
-
-            assertTrue(expKeys.remove(entry.getKey()));
+            int expKey = desc ? right - 1 - i : i;
 
-            assertEquals(new Person(entry.getKey().intValue()), all.get(i).getValue());
+            assertEquals(new Person(expKey), all.get(i).getValue());
         }
     }
 
     /** */
     private static class Person {
         /** */
+        @GridToStringInclude
         final int id;
 
         /** */
+        @GridToStringInclude
         final int descId;
 
         /** */
@@ -201,5 +198,10 @@ public class IndexQueryAliasTest extends GridCommonAbstractTest {
         @Override public int hashCode() {
             return Objects.hash(id, descId);
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Person.class, this);
+        }
     }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAllTypesTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAllTypesTest.java
index d450cbd..a98b8bc 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAllTypesTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAllTypesTest.java
@@ -48,6 +48,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
 import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
 import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
 import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
@@ -259,7 +260,7 @@ public class IndexQueryAllTypesTest extends GridCommonAbstractTest {
     /** Also checks duplicate indexed values. */
     @Test
     public void testBoolField() {
-        Function<Integer, Boolean> valGen = i -> i <= CNT / 2;
+        Function<Integer, Boolean> valGen = i -> i > CNT / 2;
 
         Function<Boolean, Person> persGen = i -> person("boolId", i);
 
@@ -272,29 +273,17 @@ public class IndexQueryAllTypesTest extends GridCommonAbstractTest {
         // All.
         check(cache.query(qry), 0, CNT, valGen, persGen);
 
-        // Lt.
+        // Eq true.
         qry = new IndexQuery<Long, Person>(Person.class, boolIdx)
-            .setCriteria(lt("boolId", true));
+            .setCriteria(eq("boolId", true));
 
         check(cache.query(qry), CNT / 2 + 1, CNT, valGen, persGen);
 
-        // Lte.
-        qry = new IndexQuery<Long, Person>(Person.class, boolIdx)
-            .setCriteria(lte("boolId", true));
-
-        check(cache.query(qry), 0, CNT, valGen, persGen);
-
-        // Gt.
+        // Eq false.
         qry = new IndexQuery<Long, Person>(Person.class, boolIdx)
-            .setCriteria(gt("boolId", false));
+            .setCriteria(eq("boolId", false));
 
         check(cache.query(qry), 0, CNT / 2 + 1, valGen, persGen);
-
-        // Gte.
-        qry = new IndexQuery<Long, Person>(Person.class, boolIdx)
-            .setCriteria(gte("boolId", false));
-
-        check(cache.query(qry), 0, CNT, valGen, persGen);
     }
 
     /** */
@@ -368,7 +357,7 @@ public class IndexQueryAllTypesTest extends GridCommonAbstractTest {
 
             assertTrue(expKeys.remove(entry.getKey()));
 
-            assertEquals(persGen.apply(valGen.apply(entry.getKey().intValue())), all.get(i).getValue());
+            assertEquals(persGen.apply(valGen.apply(left + i)), all.get(i).getValue());
         }
 
         assertTrue(expKeys.isEmpty());
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFilterTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFilterTest.java
index cf00e12..c965d67 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFilterTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFilterTest.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.cache.query;
 
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Random;
-import java.util.stream.Collectors;
+import java.util.Set;
+import java.util.TreeMap;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
@@ -31,7 +34,9 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -111,18 +116,18 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
             .setCriteria(lt("age", MAX_AGE))
             .setFilter(nameFilter);
 
-        check(cache.query(qry), nameFilter);
+        check(qry, nameFilter);
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setCriteria(lt("age", 18))
             .setFilter(nameFilter);
 
-        check(cache.query(qry), (k, v) -> v.age < 18 && nameFilter.apply(k, v));
+        check(qry, (k, v) -> v.age < 18 && nameFilter.apply(k, v));
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setFilter(nameFilter);
 
-        check(cache.query(qry), nameFilter);
+        check(qry, nameFilter);
     }
 
     /** */
@@ -134,7 +139,7 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
             .setCriteria(lt("age", MAX_AGE))
             .setFilter(ageFilter);
 
-        check(cache.query(qry), ageFilter);
+        check(qry, ageFilter);
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setCriteria(lt("age", 18))
@@ -145,7 +150,7 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setFilter(ageFilter);
 
-        check(cache.query(qry), ageFilter);
+        check(qry, ageFilter);
     }
 
     /** */
@@ -157,18 +162,18 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
             .setCriteria(lt("age", MAX_AGE))
             .setFilter(keyFilter);
 
-        check(cache.query(qry), keyFilter);
+        check(qry, keyFilter);
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setCriteria(lt("age", 18))
             .setFilter(keyFilter);
 
-        check(cache.query(qry), (k, v) -> v.age < 18 && keyFilter.apply(k, v));
+        check(qry, (k, v) -> v.age < 18 && keyFilter.apply(k, v));
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setFilter(keyFilter);
 
-        check(cache.query(qry), keyFilter);
+        check(qry, keyFilter);
     }
 
     /** */
@@ -181,12 +186,12 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
             .setCriteria(lt("age", MAX_AGE))
             .setFilter(valFilter);
 
-        check(cache.query(qry), valFilter);
+        check(qry, valFilter);
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setFilter(valFilter);
 
-        check(cache.query(qry), valFilter);
+        check(qry, valFilter);
     }
 
     /** */
@@ -202,7 +207,7 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
             .setCriteria(lt("age", 18))
             .setFilter((k, v) -> true);
 
-        check(cache.query(qry), (k, v) -> v.age < 18);
+        check(qry, (k, v) -> v.age < 18);
 
         qry = new IndexQuery<Integer, Person>(Person.class, idxName)
             .setCriteria(lt("age", MAX_AGE))
@@ -235,38 +240,73 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
     }
 
     /** */
-    private void check(QueryCursor<Cache.Entry<Integer, Person>> cursor, IgniteBiPredicate<Integer, Person> filter) {
-        Map<Integer, Person> expected = persons.entrySet().stream()
-            .filter(e -> filter.apply(e.getKey(), e.getValue()))
-            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    private void check(IndexQuery<Integer, Person> qry, IgniteBiPredicate<Integer, Person> filter) {
+        boolean pk = qry.getIndexName() == null && F.isEmpty(qry.getCriteria());
 
-        List<Cache.Entry<Integer, Person>> all = cursor.getAll();
+        TreeMap<Integer, Set<Person>> expected = pk ? pkPersons(filter) : ageIndexedPersons(filter);
 
-        assertEquals(expected.size(), all.size());
+        List<Cache.Entry<Integer, Person>> all = cache.query(qry).getAll();
 
         for (int i = 0; i < all.size(); i++) {
-            Cache.Entry<Integer, Person> entry = all.get(i);
+            Map.Entry<Integer, Set<Person>> exp = expected.firstEntry();
 
-            Person p = expected.remove(entry.getKey());
+            Cache.Entry<Integer, Person> entry = all.get(i);
 
-            assertNotNull(p);
+            assertTrue(exp.getValue().remove(entry.getValue()));
 
-            assertEquals(p, entry.getValue());
+            if (exp.getValue().isEmpty())
+                expected.remove(exp.getKey());
         }
 
         assertTrue(expected.isEmpty());
     }
 
     /** */
+    private TreeMap<Integer, Set<Person>> ageIndexedPersons(IgniteBiPredicate<Integer, Person> filter) {
+        return persons.entrySet().stream()
+            .filter(e -> filter.apply(e.getKey(), e.getValue()))
+            .collect(TreeMap::new, (m, e) -> {
+                int age = e.getValue().age;
+
+                m.computeIfAbsent(age, a -> new HashSet<>());
+
+                m.get(age).add(e.getValue());
+
+            }, (l, r) -> {
+                r.forEach((k, v) -> {
+                    int age = ((Person)v).age;
+
+                    l.computeIfAbsent(age, a -> new HashSet<>());
+
+                    l.get(age).add((Person)v);
+                });
+            });
+    }
+
+    /** */
+    private TreeMap<Integer, Set<Person>> pkPersons(IgniteBiPredicate<Integer, Person> filter) {
+        return persons.entrySet().stream()
+            .filter(e -> filter.apply(e.getKey(), e.getValue()))
+            .collect(
+                TreeMap::new,
+                (m, e) -> m.put(e.getKey(), new HashSet<>(Collections.singleton(e.getValue()))),
+                TreeMap::putAll
+            );
+    }
+
+    /** */
     private static class Person {
         /** */
+        @GridToStringInclude
         final int id;
 
         /** */
+        @GridToStringInclude
         @QuerySqlField(orderedGroups = @QuerySqlField.Group(name = IDX, order = 0))
         final int age;
 
         /** */
+        @GridToStringInclude
         final String name;
 
         /** */
@@ -293,5 +333,10 @@ public class IndexQueryFilterTest extends GridCommonAbstractTest {
         @Override public int hashCode() {
             return Objects.hash(id, age, name);
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Person.class, this);
+        }
     }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryKeepBinaryTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryKeepBinaryTest.java
index e37971a..fa4e7f9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryKeepBinaryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryKeepBinaryTest.java
@@ -19,9 +19,7 @@ package org.apache.ignite.cache.query;
 
 import java.util.List;
 import java.util.Objects;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.LongStream;
+import java.util.Random;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -97,6 +95,35 @@ public class IndexQueryKeepBinaryTest extends GridCommonAbstractTest {
         check(cache.withKeepBinary().query(qry), CNT / 4 + 1, CNT / 2);
     }
 
+    /** */
+    @Test
+    public void testComplexSqlPrimaryKey() {
+        String valType = "MY_VALUE_TYPE";
+        String tblCacheName = "MY_TABLE_CACHE";
+
+        SqlFieldsQuery qry = new SqlFieldsQuery("create table my_table (id1 int, id2 int, id3 int," +
+            " PRIMARY KEY(id1, id2)) with \"VALUE_TYPE=" + valType + ",CACHE_NAME=" + tblCacheName + "\";");
+
+        cache.query(qry);
+
+        qry = new SqlFieldsQuery("insert into my_table(id1, id2, id3) values(?, ?, ?);");
+
+        for (int i = 0; i < CNT; i++) {
+            qry.setArgs(i, i, i);
+
+            cache.query(qry);
+        }
+
+        int pivot = new Random().nextInt(CNT);
+
+        IgniteCache<BinaryObject, BinaryObject> tblCache = grid(0).cache(tblCacheName);
+
+        IndexQuery<BinaryObject, BinaryObject> idxQry = new IndexQuery<BinaryObject, BinaryObject>(valType)
+            .setCriteria(lt("id1", pivot));
+
+        checkBinary(tblCache.withKeepBinary().query(idxQry), 0, pivot);
+    }
+
     /**
      * @param left First cache key, inclusive.
      * @param right Last cache key, exclusive.
@@ -106,19 +133,30 @@ public class IndexQueryKeepBinaryTest extends GridCommonAbstractTest {
 
         assertEquals(right - left, all.size());
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
-
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Long, ?> entry = all.get(i);
 
-            assertTrue(expKeys.remove(entry.getKey()));
+            assertEquals(left + i, entry.getKey().intValue());
 
             BinaryObject o = all.get(i).getValue();
 
             assertEquals(new Person(entry.getKey().intValue()), o.deserialize());
         }
+    }
+
+    /** */
+    private void checkBinary(QueryCursor cursor, int left, int right) {
+        List<Cache.Entry<BinaryObject, BinaryObject>> all = cursor.getAll();
 
-        assertTrue(expKeys.isEmpty());
+        assertEquals(right - left, all.size());
+
+        for (int i = 0; i < all.size(); i++) {
+            Cache.Entry<BinaryObject, BinaryObject> entry = all.get(i);
+
+            assertEquals(left + i, (int)entry.getKey().field("id1"));
+            assertEquals(left + i, (int)entry.getKey().field("id2"));
+            assertEquals(left + i, (int)entry.getValue().field("id3"));
+        }
     }
 
     /** */
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java
index 7bc90f7..d8db058 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java
@@ -23,9 +23,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.LongStream;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -180,13 +177,13 @@ public class IndexQueryQueryEntityTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", pivot));
 
-        check(cache.query(qry), 0, pivot);
+        check(cache.query(qry), 0, pivot, false);
 
         // Lt, desc index.
         IndexQuery<Long, Person> descQry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
             .setCriteria(lt("descId", pivot));
 
-        check(cache.query(descQry), 0, pivot);
+        check(cache.query(descQry), 0, pivot, true);
     }
 
     /** */
@@ -200,30 +197,30 @@ public class IndexQueryQueryEntityTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", pivot));
 
-        check(cacheTblName.query(qry), 0, pivot);
+        check(cacheTblName.query(qry), 0, pivot, false);
 
         // Lt, desc index.
         IndexQuery<Long, Person> descQry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
             .setCriteria(lt("descId", pivot));
 
-        check(cacheTblName.query(descQry), 0, pivot);
+        check(cacheTblName.query(descQry), 0, pivot, true);
     }
 
     /**
      * @param left First cache key, inclusive.
      * @param right Last cache key, exclusive.
      */
-    private void check(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {
+    private void check(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right, boolean desc) {
         List<Cache.Entry<Long, Person>> all = cursor.getAll();
 
         assertEquals(right - left, all.size());
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
-
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Long, Person> entry = all.get(i);
 
-            assertTrue(expKeys.remove(entry.getKey()));
+            int exp = desc ? right - i - 1 : left + i;
+
+            assertEquals(exp, entry.getKey().intValue());
 
             assertEquals(new Person(entry.getKey().intValue()), all.get(i).getValue());
         }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryRangeTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryRangeTest.java
index 31dcfa7..aca4efd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryRangeTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryRangeTest.java
@@ -17,14 +17,17 @@
 
 package org.apache.ignite.cache.query;
 
-import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
 import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.LongStream;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.IntUnaryOperator;
+import java.util.stream.Stream;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -35,6 +38,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -70,9 +74,6 @@ public class IndexQueryRangeTest extends GridCommonAbstractTest {
     private Ignite crd;
 
     /** */
-    private int duplicates;
-
-    /** */
     private IgniteCache<Long, Person> cache;
 
     /** */
@@ -96,40 +97,31 @@ public class IndexQueryRangeTest extends GridCommonAbstractTest {
     public int backups;
 
     /** */
-    @Parameterized.Parameters(name = "qryPar={0} atomicity={1} mode={2} node={3} backups={4}")
+    @Parameterized.Parameter(5)
+    public String idxName;
+
+    /** Number of duplicates of indexed value. */
+    @Parameterized.Parameter(6)
+    public int duplicates;
+
+    /** */
+    @Parameterized.Parameters(name = "qryPar={0} atomicity={1} mode={2} node={3} backups={4} idxName={5} duplicates={6}")
     public static Collection<Object[]> testParams() {
-        return Arrays.asList(
-            new Object[] {1, TRANSACTIONAL, REPLICATED, "CRD", 0},
-            new Object[] {1, TRANSACTIONAL, PARTITIONED, "CRD", 0},
-            new Object[] {4, TRANSACTIONAL, PARTITIONED, "CRD", 0},
-
-            new Object[] {1, ATOMIC, REPLICATED, "CRD", 0},
-            new Object[] {1, ATOMIC, PARTITIONED, "CRD", 0},
-            new Object[] {4, ATOMIC, PARTITIONED, "CRD", 0},
-
-            new Object[] {1, TRANSACTIONAL, REPLICATED, "CLN", 0},
-            new Object[] {1, TRANSACTIONAL, PARTITIONED, "CLN", 0},
-            new Object[] {4, TRANSACTIONAL, PARTITIONED, "CLN", 0},
-
-            new Object[] {1, ATOMIC, REPLICATED, "CLN", 0},
-            new Object[] {1, ATOMIC, PARTITIONED, "CLN", 0},
-            new Object[] {4, ATOMIC, PARTITIONED, "CLN", 0},
-
-            new Object[] {1, TRANSACTIONAL, REPLICATED, "CRD", 2},
-            new Object[] {1, TRANSACTIONAL, PARTITIONED, "CRD", 2},
-            new Object[] {4, TRANSACTIONAL, PARTITIONED, "CRD", 2},
-
-            new Object[] {1, ATOMIC, REPLICATED, "CRD", 2},
-            new Object[] {1, ATOMIC, PARTITIONED, "CRD", 2},
-            new Object[] {4, ATOMIC, PARTITIONED, "CRD", 2},
-
-            new Object[] {1, TRANSACTIONAL, REPLICATED, "CLN", 2},
-            new Object[] {1, TRANSACTIONAL, PARTITIONED, "CLN", 2},
-            new Object[] {4, TRANSACTIONAL, PARTITIONED, "CLN", 2},
-
-            new Object[] {1, ATOMIC, REPLICATED, "CLN", 2},
-            new Object[] {1, ATOMIC, PARTITIONED, "CLN", 2},
-            new Object[] {4, ATOMIC, PARTITIONED, "CLN", 2});
+        List<Object[]> params = new ArrayList<>();
+
+        Stream.of("CRD", "CLN").forEach(node ->
+            Stream.of(0, 2).forEach(backups ->
+                Stream.of(1, 10).forEach(duplicates ->
+                    Stream.of(IDX, DESC_IDX).forEach(idx -> {
+                        params.add(new Object[] {1, TRANSACTIONAL, REPLICATED, node, backups, idx, duplicates});
+                        params.add(new Object[] {1, TRANSACTIONAL, PARTITIONED, node, backups, idx, duplicates});
+                        params.add(new Object[] {4, TRANSACTIONAL, PARTITIONED, node, backups, idx, duplicates});
+                    })
+                )
+            )
+        );
+
+        return params;
     }
 
     /** {@inheritDoc} */
@@ -172,174 +164,104 @@ public class IndexQueryRangeTest extends GridCommonAbstractTest {
 
     /** */
     @Test
-    public void testRangeQueries() {
-        duplicates = 1;
-
-        checkRangeQueries();
-    }
-
-    /** */
-    @Test
-    public void testRangeDescQueries() {
-        duplicates = 1;
-
-        checkRangeDescQueries();
-    }
-
-    /** */
-    @Test
-    public void testRangeQueriesWithDuplicatedData() {
-        duplicates = 10;
-
-        checkRangeQueries();
-    }
-
-    /** */
-    @Test
-    public void testRangeDescQueriesWithDuplicatedData() {
-        duplicates = 10;
-
-        checkRangeDescQueries();
-    }
-
-    /** */
-    public void checkRangeQueries() {
+    public void testRangeQueries() throws Exception {
         // Query empty cache.
-        IndexQuery<Long, Person> qry = new IndexQuery<>(Person.class, IDX);
+        IndexQuery<Long, Person> qry = new IndexQuery<>(Person.class, idxName);
 
         assertTrue(cache.query(qry).getAll().isEmpty());
 
         // Add data
         insertData();
 
-        qry = new IndexQuery<>(Person.class, IDX);
+        qry = new IndexQuery<>(Person.class, idxName);
 
-        check(cache.query(qry), 0, CNT);
+        check(qry, 0, CNT);
 
         // Range queries.
+        String fld = idxName.equals(IDX) ? "id" : "descId";
+
         int pivot = new Random().nextInt(CNT);
 
         // Eq.
-        qry = new IndexQuery<Long, Person>(Person.class, IDX)
-            .setCriteria(eq("id", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(eq(fld, pivot));
 
-        check(cache.query(qry), pivot, pivot + 1);
+        check(qry, pivot, pivot + 1);
 
         // Lt.
-        qry = new IndexQuery<Long, Person>(Person.class, IDX)
-            .setCriteria(lt("id", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lt(fld, pivot));
 
-        check(cache.query(qry), 0, pivot);
+        check(qry, 0, pivot);
 
         // Lte.
-        qry = new IndexQuery<Long, Person>(Person.class, IDX)
-            .setCriteria(lte("id", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lte(fld, pivot));
 
-        check(cache.query(qry), 0, pivot + 1);
+        check(qry, 0, pivot + 1);
 
         // Gt.
-        qry = new IndexQuery<Long, Person>(Person.class, IDX)
-            .setCriteria(gt("id", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gt(fld, pivot));
 
-        check(cache.query(qry), pivot + 1, CNT);
+        check(qry, pivot + 1, CNT);
 
         // Gte.
-        qry = new IndexQuery<Long, Person>(Person.class, IDX)
-            .setCriteria(gte("id", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gte(fld, pivot));
 
-        check(cache.query(qry), pivot, CNT);
+        check(qry, pivot, CNT);
 
         // Between.
         int lower = new Random().nextInt(CNT / 2);
         int upper = lower + CNT / 20;
 
-        qry = new IndexQuery<Long, Person>(Person.class, IDX)
-            .setCriteria(between("id", lower, upper));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(between(fld, lower, upper));
 
-        check(cache.query(qry), lower, upper + 1);
-    }
-
-    /** */
-    public void checkRangeDescQueries() {
-        // Query empty cache.
-        IndexQuery<Long, Person> qry = new IndexQuery<>(Person.class, DESC_IDX);
-
-        assertTrue(cache.query(qry).getAll().isEmpty());
-
-        // Add data
-        insertData();
-
-        qry = new IndexQuery<>(Person.class, DESC_IDX);
-
-        check(cache.query(qry), 0, CNT);
-
-        // Range queries.
-        int pivot = new Random().nextInt(CNT);
-
-        // Eq.
-        qry = new IndexQuery<Long, Person>(Person.class, DESC_IDX)
-            .setCriteria(eq("descId", pivot));
-
-        check(cache.query(qry), pivot, pivot + 1);
-
-        // Lt, desc index.
-        IndexQuery<Long, Person> descQry = new IndexQuery<Long, Person>(Person.class, DESC_IDX)
-            .setCriteria(lt("descId", pivot));
-
-        check(cache.query(descQry), 0, pivot);
-
-        // Lte, desc index.
-        descQry = new IndexQuery<Long, Person>(Person.class, DESC_IDX)
-            .setCriteria(lte("descId", pivot));
-
-        check(cache.query(descQry), 0, pivot + 1);
-
-        // Gt, desc index.
-        descQry = new IndexQuery<Long, Person>(Person.class, DESC_IDX)
-            .setCriteria(gt("descId", pivot));
-
-        check(cache.query(descQry), pivot + 1, CNT);
-
-        // Gte, desc index.
-        descQry = new IndexQuery<Long, Person>(Person.class, DESC_IDX)
-            .setCriteria(gte("descId", pivot));
-
-        check(cache.query(descQry), pivot, CNT);
-
-        // Between, desc index.
-        int lower = new Random().nextInt(CNT / 2);
-        int upper = lower + CNT / 20;
-
-        descQry = new IndexQuery<Long, Person>(Person.class, DESC_IDX)
-            .setCriteria(between("descId", lower, upper));
-
-        check(cache.query(descQry), lower, upper + 1);
+        check(qry, lower, upper + 1);
     }
 
     /**
      * @param left First cache key, inclusive.
      * @param right Last cache key, exclusive.
      */
-    private void check(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {
-        List<Cache.Entry<Long, Person>> all = cursor.getAll();
+    private void check(Query<Cache.Entry<Long, Person>> qry, int left, int right) throws Exception {
+        QueryCursor<Cache.Entry<Long, Person>> cursor = cache.query(qry);
 
-        assertFalse(all.isEmpty());
+        int expSize = (right - left) * duplicates;
 
-        assertEquals((right - left) * duplicates, all.size());
+        Set<Long> expKeys = new HashSet<>(expSize);
+        List<Integer> expOrderedValues = new LinkedList<>();
 
-        Set<Long> expKeys = LongStream
-            .range((long) left * duplicates, (long) right * duplicates).boxed()
-            .collect(Collectors.toSet());
+        boolean desc = idxName.equals(DESC_IDX);
 
-        for (int i = 0; i < all.size(); i++) {
-            Cache.Entry<Long, Person> entry = all.get(i);
+        int from = desc ? right - 1 : left;
+        int to = desc ? left - 1 : right;
+        IntUnaryOperator op = (i) -> desc ? i - 1 : i + 1;
+
+        for (int i = from; i != to; i = op.applyAsInt(i)) {
+            for (int j = 0; j < duplicates; j++) {
+                expOrderedValues.add(i);
+                expKeys.add((long) CNT * j + i);
+            }
+        }
+
+        AtomicInteger actSize = new AtomicInteger();
+
+        ((QueryCursorEx<Cache.Entry<Long, Person>>)cursor).getAll(entry -> {
+            assertEquals(expOrderedValues.remove(0).intValue(), entry.getValue().id);
 
             assertTrue(expKeys.remove(entry.getKey()));
 
-            int persId = entry.getKey().intValue() / duplicates;
+            int persId = entry.getKey().intValue() % CNT;
 
-            assertEquals(new Person(persId), all.get(i).getValue());
-        }
+            assertEquals(new Person(persId), entry.getValue());
+
+            actSize.incrementAndGet();
+        });
+
+        assertEquals(expSize, actSize.get());
 
         assertTrue(expKeys.isEmpty());
     }
@@ -350,7 +272,7 @@ public class IndexQueryRangeTest extends GridCommonAbstractTest {
             for (int persId = 0; persId < CNT; persId++) {
                 // Create duplicates of data.
                 for (int i = 0; i < duplicates; i++)
-                    streamer.addData((long) persId * duplicates + i, new Person(persId));
+                    streamer.addData((long) CNT * i + persId, new Person(persId));
             }
         }
     }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
index 1825f86..54fb289 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
@@ -22,15 +22,13 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.LongStream;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
@@ -83,7 +81,7 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        crd = startGrids(1);
+        crd = startGrids(2);
 
         cache = crd.createCache(new CacheConfiguration<>().setName(CACHE));
     }
@@ -150,16 +148,16 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class.getName(), qryDescIdxName)
             .setCriteria(lt("descId", pivot));
 
-        check(tblCache.query(qry), 0, pivot);
+        check(qry, 0, pivot);
 
         qry = new IndexQuery<Long, Person>(Person.class.getName(), qryDescIdxName)
             .setCriteria(lt("DESCID", pivot));
 
-        check(tblCache.query(qry), 0, pivot);
+        check(qry, 0, pivot);
 
         qry = new IndexQuery<>(Person.class.getName(), qryDescIdxName);
 
-        check(tblCache.query(qry), 0, CNT);
+        check(qry, 0, CNT);
     }
 
     /** Should support only original field. */
@@ -174,7 +172,7 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class.getName(), qryDescIdxName)
             .setCriteria(lt("descId", pivot));
 
-        check(tblCache.query(qry), 0, pivot);
+        check(qry, 0, pivot);
 
         String errMsg = qryDescIdxName != null ? "Index doesn't match criteria." : "No index found for criteria.";
 
@@ -201,7 +199,7 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class.getName(), idx)
             .setCriteria(lt("descId", pivot));
 
-        check(tblCache.query(qry), 0, pivot);
+        check(qry, 0, pivot);
 
         if (qryDescIdxName != null) {
             GridTestUtils.assertThrowsAnyCause(null, () -> {
@@ -256,24 +254,26 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class.getName(), qryDescIdxName)
             .setCriteria(eq("_KEY", (long) pivot), lte("descId", pivot));
 
-        check(tblCache.query(qry), pivot, pivot + 1);
+        check(qry, pivot, pivot + 1);
     }
 
     /**
      * @param left First cache key, inclusive.
      * @param right Last cache key, exclusive.
      */
-    private void check(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {
-        List<Cache.Entry<Long, Person>> all = cursor.getAll();
+    private void check(IndexQuery<Long, Person> qry, int left, int right) {
+        boolean pk = qry.getIndexName() == null && F.isEmpty(qry.getCriteria());
 
-        assertEquals(right - left, all.size());
+        List<Cache.Entry<Long, Person>> all = tblCache.query(qry).getAll();
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
+        assertEquals(right - left, all.size());
 
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Long, Person> entry = all.get(i);
 
-            assertTrue(expKeys.remove(entry.getKey()));
+            int exp = pk ? left + i : right - i - 1;
+
+            assertEquals(exp, entry.getKey().intValue());
 
             assertEquals(new Person(entry.getKey().intValue()), all.get(i).getValue());
         }
@@ -288,13 +288,10 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
 
         assertEquals(right - left, all.size());
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
-
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Long, BinaryObject> entry = all.get(i);
 
-            assertTrue(expKeys.remove(entry.getKey()));
-
+            assertEquals(right - 1 - i, entry.getKey().intValue());
             assertEquals(entry.getKey().intValue(), (int) entry.getValue().field("id"));
             assertEquals(entry.getKey().intValue(), (int) entry.getValue().field("descId"));
         }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java
index 20c7db4..3dd7578 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java
@@ -260,12 +260,10 @@ public class MultiTableIndexQuery extends GridCommonAbstractTest {
 
         assertEquals(right - left, all.size());
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
-
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Long, SecondPerson> entry = all.get(i);
 
-            assertTrue(expKeys.remove(entry.getKey()));
+            assertEquals(left + i, entry.getKey().intValue());
 
             assertEquals(new SecondPerson(entry.getKey().intValue()), all.get(i).getValue());
         }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java
index 931fffb..3064414 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java
@@ -132,7 +132,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryKeyPKIdx)
             .setCriteria(lt("_KEY", (long) pivot));
 
-        checkPerson(cache.query(qry), 0, pivot);
+        checkPerson(qry, 0, pivot, false);
     }
 
     /** */
@@ -190,7 +190,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", 1));
 
-        checkPerson(cache.query(qry), 0, CNT);
+        checkPerson(qry, 0, CNT, false);
 
         // Checks the same with query for DESC_IDX.
         qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
@@ -201,7 +201,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
             .setCriteria(lt("id", 1));
 
-        checkPerson(cache.query(qry), 0, CNT);
+        checkPerson(qry, 0, CNT, qryDescIdx != null);
     }
 
     /** */
@@ -221,7 +221,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", 1), lt("secId", CNT));
 
-        checkPerson(cache.query(qry), 0, CNT);
+        checkPerson(qry, 0, CNT, false);
 
         // Should return part of data, as ID equals to inserted data ID field.
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
@@ -233,7 +233,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", 1), lt("secId", pivot));
 
-        checkPerson(cache.query(qry), 0, pivot);
+        checkPerson(qry, 0, pivot, false);
     }
 
     /** */
@@ -253,7 +253,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("secId", CNT), lt("id", 1));
 
-        checkPerson(cache.query(qry), 0, CNT);
+        checkPerson(qry, 0, CNT, false);
 
         // Should return part of data, as ID equals to inserted data ID field.
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
@@ -265,264 +265,145 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("secId", pivot), lt("id", 1));
 
-        checkPerson(cache.query(qry), 0, pivot);
+        checkPerson(qry, 0, pivot, false);
     }
 
     /** */
     @Test
-    public void testLegalDifferentCriteria() {
+    public void testLegalDifferentCriteriaAscIndex() {
+        testLegalDifferentCriteria(qryIdx, "secId", false);
+    }
+
+    /** */
+    @Test
+    public void testLegalDifferentCriteriaWithDescIdx() {
+        testLegalDifferentCriteria(qryDescIdx, "descId", true);
+    }
+
+    /** */
+    public void testLegalDifferentCriteria(String idxName, String fldName, boolean desc) {
         insertData();
 
         int pivot = new Random().nextInt(CNT);
 
         // Eq as first criterion.
-        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(eq("id", 1), lt("secId", pivot));
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(eq("id", 1), lt(fldName, pivot));
 
         assertTrue(cache.query(qry).getAll().isEmpty());
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(eq("id", 0), lte("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(eq("id", 0), lte(fldName, pivot));
 
-        checkPerson(cache.query(qry), 0, pivot + 1);
+        checkPerson(qry, 0, pivot + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(eq("id", 0), gt("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(eq("id", 0), gt(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot + 1, CNT);
+        checkPerson(qry, pivot + 1, CNT, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(eq("id", 0), gte("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(eq("id", 0), gte(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, CNT);
+        checkPerson(qry, pivot, CNT, desc);
 
         int lower = new Random().nextInt(CNT / 2);
         int upper = lower + new Random().nextInt(CNT / 2);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(eq("id", 0), between("secId", lower, upper));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(eq("id", 0), between(fldName, lower, upper));
 
-        checkPerson(cache.query(qry), lower, upper + 1);
+        checkPerson(qry, lower, upper + 1, desc);
 
         // Lt as first criterion.
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(lt("id", 1), lte("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lt("id", 1), lte(fldName, pivot));
 
-        checkPerson(cache.query(qry), 0, pivot + 1);
+        checkPerson(qry, 0, pivot + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(lt("id", 1), eq("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lt("id", 1), eq(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, pivot + 1);
+        checkPerson(qry, pivot, pivot + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(lt("id", 1), between("secId", lower, upper));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lt("id", 1), between(fldName, lower, upper));
 
-        checkPerson(cache.query(qry), lower, upper + 1);
+        checkPerson(qry, lower, upper + 1, desc);
 
         // Lte as first criterion.
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(lte("id", 0), lt("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lte("id", 0), lt(fldName, pivot));
 
-        checkPerson(cache.query(qry), 0, pivot);
+        checkPerson(qry, 0, pivot, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(lte("id", 1), between("secId", lower, upper));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lte("id", 1), between(fldName, lower, upper));
 
-        checkPerson(cache.query(qry), lower, upper + 1);
+        checkPerson(qry, lower, upper + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(lte("id", 0), eq("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(lte("id", 0), eq(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, pivot + 1);
+        checkPerson(qry, pivot, pivot + 1, desc);
 
         // Gt as first criterion.
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(gt("id", -1), gte("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gt("id", -1), gte(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, CNT);
+        checkPerson(qry, pivot, CNT, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(gt("id", -1), eq("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gt("id", -1), eq(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, pivot + 1);
+        checkPerson(qry, pivot, pivot + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(gt("id", -1), between("secId", lower, upper));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gt("id", -1), between(fldName, lower, upper));
 
-        checkPerson(cache.query(qry), lower, upper + 1);
+        checkPerson(qry, lower, upper + 1, desc);
 
         // Gte as first criterion.
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(gte("id", 0), gt("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gte("id", 0), gt(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot + 1, CNT);
+        checkPerson(qry, pivot + 1, CNT, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(gte("id", 0), between("secId", lower, upper));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gte("id", 0), between(fldName, lower, upper));
 
-        checkPerson(cache.query(qry), lower, upper + 1);
+        checkPerson(qry, lower, upper + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(gte("id", 0), eq("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(gte("id", 0), eq(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, pivot + 1);
+        checkPerson(qry, pivot, pivot + 1, desc);
 
         // Between as first criterion.
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(between("id", -1, 1), lt("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(between("id", -1, 1), lt(fldName, pivot));
 
-        checkPerson(cache.query(qry), 0, pivot);
+        checkPerson(qry, 0, pivot, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(between("id", -1, 1), lte("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(between("id", -1, 1), lte(fldName, pivot));
 
-        checkPerson(cache.query(qry), 0, pivot + 1);
+        checkPerson(qry, 0, pivot + 1, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(between("id", -1, 1), gt("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(between("id", -1, 1), gt(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot + 1, CNT);
+        checkPerson(qry, pivot + 1, CNT, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(between("id", -1, 1), gte("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(between("id", -1, 1), gte(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, CNT);
+        checkPerson(qry, pivot, CNT, desc);
 
-        qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
-            .setCriteria(between("id", -1, 1), eq("secId", pivot));
+        qry = new IndexQuery<Long, Person>(Person.class, idxName)
+            .setCriteria(between("id", -1, 1), eq(fldName, pivot));
 
-        checkPerson(cache.query(qry), pivot, pivot + 1);
-    }
-
-    /** */
-    @Test
-    public void testLegalDifferentCriteriaWithDescIdx() {
-        insertData();
-
-        int pivot = new Random().nextInt(CNT);
-        int lower = new Random().nextInt(CNT / 2);
-        int upper = lower + new Random().nextInt(CNT / 2);
-
-        // Eq as first criteria.
-        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(eq("id", 1), lt("descId", pivot));
-
-        assertTrue(cache.query(qry).getAll().isEmpty());
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(eq("id", 0), lt("descId", pivot));
-
-        checkPerson(cache.query(qry), 0, pivot);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(eq("id", 0), lte("descId", pivot));
-
-        checkPerson(cache.query(qry), 0, pivot + 1);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(eq("id", 0), gt("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot + 1, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(eq("id", 0), gte("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(eq("id", 0), between("descId", lower, upper));
-
-        checkPerson(cache.query(qry), lower, upper + 1);
-
-        // Lt as first criteria.
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lt("id", 1), gt("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot + 1, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lt("id", 1), gte("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lt("id", 1), between("descId", lower, upper));
-
-        checkPerson(cache.query(qry), lower, upper + 1);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lt("id", 1), eq("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, pivot + 1);
-
-        // Lte as first criteria.
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lte("id", 0), gt("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot + 1, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lte("id", 0), gte("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lte("id", 0), between("descId", lower, upper));
-
-        checkPerson(cache.query(qry), lower, upper + 1);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(lte("id", 0), eq("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, pivot + 1);
-
-        // Gte as first criteria.
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(gte("id", 0), lt("descId", pivot));
-
-        checkPerson(cache.query(qry), 0, pivot);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(gte("id", 0), lte("descId", pivot));
-
-        checkPerson(cache.query(qry), 0, pivot + 1);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(gte("id", 0), between("descId", lower, upper));
-
-        checkPerson(cache.query(qry), lower, upper + 1);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(gte("id", 0), eq("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, pivot + 1);
-
-        // Between as first criteria.
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(between("id", -1, 1), lt("descId", pivot));
-
-        checkPerson(cache.query(qry), 0, pivot);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(between("id", -1, 1), lte("descId", pivot));
-
-        checkPerson(cache.query(qry), 0, pivot + 1);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(between("id", -1, 1), gt("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot + 1, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(between("id", -1, 1), gte("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, CNT);
-
-        qry = new IndexQuery<Long, Person>(Person.class, qryDescIdx)
-            .setCriteria(between("id", -1, 1), eq("descId", pivot));
-
-        checkPerson(cache.query(qry), pivot, pivot + 1);
+        checkPerson(qry, pivot, pivot + 1, desc);
     }
 
     /** */
@@ -535,12 +416,12 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", 1), gt("secId", pivot));
 
-        checkPerson(cache.query(qry), pivot + 1, CNT);
+        checkPerson(qry, pivot + 1, CNT, false);
 
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(lt("id", 1), gte("secId", pivot));
 
-        checkPerson(cache.query(qry), pivot, CNT);
+        checkPerson(qry, pivot, CNT, false);
 
         qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(gt("id", 2), lt("secId", pivot));
@@ -579,7 +460,7 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
             .setCriteria(eq("id", 0), lt("secId", pivot), lt("_KEY", (long) pivot));
 
-        checkPerson(cache.query(qry), 0, pivot);
+        checkPerson(qry, 0, pivot, false);
     }
 
     /** */
@@ -591,17 +472,25 @@ public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
     }
 
     /** */
-    private void checkPerson(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {
-        List<Cache.Entry<Long, Person>> all = cursor.getAll();
+    private void checkPerson(IndexQuery<Long, Person> qry, int left, int right, boolean desc) {
+        boolean fullSort = qry.getCriteria().size() == 2;
+
+        List<Cache.Entry<Long, Person>> all = cache.query(qry).getAll();
 
         assertEquals(right - left, all.size());
 
-        Set<Long> expKeys = LongStream.range(left, right).boxed().collect(Collectors.toSet());
+        Set<Long> expKeys = fullSort ? null : LongStream.range(left, right).boxed().collect(Collectors.toSet());
 
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Long, Person> entry = all.get(i);
 
-            assertTrue(expKeys.remove(entry.getKey()));
+            if (fullSort) {
+                int exp = desc ? right - 1 - i : left + i;
+
+                assertEquals(exp, entry.getKey().intValue());
+            }
+            else
+                assertTrue(expKeys.remove(entry.getKey()));
 
             assertEquals(new Person(entry.getKey().intValue()), all.get(i).getValue());
         }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
index fff71a2..ce39736 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
@@ -19,8 +19,8 @@ package org.apache.ignite.cache.query;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
-import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import javax.cache.Cache;
@@ -232,15 +232,15 @@ public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
 
         assertEquals(errMsg, right - left, all.size());
 
-        Set<Integer> expKeys = IntStream.range(left, right).boxed().collect(Collectors.toSet());
+        boolean desc = Objects.equals(idxName, DESC_ID_IDX);
 
         for (int i = 0; i < all.size(); i++) {
             Cache.Entry<Integer, Person> entry = all.get(i);
 
-            assertTrue(errMsg, expKeys.remove(entry.getKey()));
-        }
+            int exp = desc ? right - 1 - i : left + i;
 
-        assertTrue(errMsg, expKeys.isEmpty());
+            assertEquals(errMsg, exp, entry.getKey().intValue());
+        }
     }
 
     /** */
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
index ad0354d..1b91f77 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
@@ -90,7 +90,8 @@ public class GridCacheFullTextQueryPagesTest extends GridCacheFullTextQueryAbstr
     public void testTextQueryMultiplePagesNoLimit() {
         checkTextQuery("1*", 0, PAGE_SIZE);
 
-        checkPages(4, 8, 0);
+        // Send 2 additional load requests on each node.
+        checkPages(NODES, NODES * 2, 0);
     }
 
     /** Test that do not send cache page request after limit exceeded. */
@@ -98,7 +99,9 @@ public class GridCacheFullTextQueryPagesTest extends GridCacheFullTextQueryAbstr
     public void testTextQueryLimitedMultiplePages() {
         checkTextQuery("1*", QUERY_LIMIT, 30);
 
-        checkPages(4, 7, 3);
+        // We hold 2 pre-loaded pages per node. Then we send additional load request for every node on beginning,
+        // and 2 more while iterating over data and finish preloaded pages (depends on page size).
+        checkPages(NODES, NODES + 2, NODES);
     }
 
     /** Test that rerequest some pages but then send a cancel query after limit exceeded. */
@@ -106,7 +109,9 @@ public class GridCacheFullTextQueryPagesTest extends GridCacheFullTextQueryAbstr
     public void testTextQueryHighLimitedMultiplePages() {
         checkTextQuery("1*", QUERY_LIMIT, 20);
 
-        checkPages(4, 8, 3);
+        // We hold 2 pre-loaded pages per node. Then we send additional load request for every node on beginning,
+        // and 4 more while iterating over data and finish preloaded pages (depends on page size).
+        checkPages(NODES, NODES + 4, NODES);
     }
 
     /** */