You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by av...@apache.org on 2021/10/19 12:47:54 UTC

[ignite] branch ignite-2.12 updated: IGNITE-15535 Allow specify multiple criteria for the same field (#9429) (#9500)

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

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


The following commit(s) were added to refs/heads/ignite-2.12 by this push:
     new d4f5efb  IGNITE-15535 Allow specify multiple criteria for the same field (#9429) (#9500)
d4f5efb is described below

commit d4f5efbac693f63eb325e35498335f1a6c84fc6d
Author: Maksim Timonin <ti...@gmail.com>
AuthorDate: Tue Oct 19 15:47:30 2021 +0300

    IGNITE-15535 Allow specify multiple criteria for the same field (#9429) (#9500)
---
 .../org/apache/ignite/cache/query/IndexQuery.java  |   8 -
 .../cache/query/RangeIndexQueryCriterion.java      |   5 +
 .../cache/query/index/IndexDefinition.java         |   6 +-
 .../cache/query/index/IndexQueryProcessor.java     | 451 ++++++++++++++-------
 .../query/index/sorted/IndexKeyDefinition.java     |  11 +-
 .../query/index/sorted/IndexRowComparator.java     |  12 +-
 .../query/index/sorted/IndexRowCompartorImpl.java  |  16 +-
 .../query/index/sorted/inline/InlineIndexImpl.java |   2 +-
 .../sorted/inline/InlineIndexKeyTypeRegistry.java  |   5 +-
 .../query/index/sorted/inline/InlineIndexTree.java |  10 +-
 .../sorted/inline/InlineObjectBytesDetector.java   |  13 +-
 .../index/sorted/inline/InlineRecommender.java     |   4 +-
 .../apache/ignite/testframework/GridTestUtils.java |  20 +-
 .../query/h2/opt/GeoSpatialIndexDefinition.java    |  13 +-
 .../processors/query/h2/opt/GeoSpatialUtils.java   |   7 +-
 .../processors/query/h2/index/H2RowComparator.java |  21 +-
 .../query/h2/index/QueryIndexDefinition.java       |   5 +-
 .../h2/index/QueryIndexKeyDefinitionProvider.java  |  18 +-
 .../query/h2/index/QueryIndexRowHandler.java       |   7 +-
 .../query/h2/index/QueryRowHandlerFactory.java     |   5 +-
 .../h2/index/client/ClientIndexDefinition.java     |   5 +-
 .../query/h2/index/client/ClientIndexFactory.java  |   9 +-
 .../processors/query/h2/opt/GridH2Table.java       |   3 +-
 .../ignite/cache/query/IndexQueryFailoverTest.java |  43 +-
 .../ignite/cache/query/IndexQuerySqlIndexTest.java |   4 +-
 .../ignite/cache/query/IndexQueryTestSuite.java    |   3 +-
 .../cache/query/RepeatedFieldIndexQueryTest.java   | 360 ++++++++++++++++
 27 files changed, 827 insertions(+), 239 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
index 8854037..25f7807 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
@@ -20,9 +20,7 @@ package org.apache.ignite.cache.query;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import javax.cache.Cache;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -178,16 +176,10 @@ public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
 
         Class<?> critCls = criteria.get(0).getClass();
 
-        Set<String> fields = new HashSet<>();
-
         for (IndexQueryCriterion c: criteria) {
             A.notNull(c, "criteria");
             A.ensure(c.getClass() == critCls,
                 "Expect a the same criteria class for merging criteria. Exp=" + critCls + ", act=" + c.getClass());
-
-            A.ensure(!fields.contains(c.field()), "Duplicated field in criteria: " + c.field() + ".");
-
-            fields.add(c.field());
         }
 
         this.criteria = Collections.unmodifiableList(criteria);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/RangeIndexQueryCriterion.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/RangeIndexQueryCriterion.java
index 7dd9150..1c5dec4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/RangeIndexQueryCriterion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/RangeIndexQueryCriterion.java
@@ -120,4 +120,9 @@ public final class RangeIndexQueryCriterion implements IndexQueryCriterion {
     @Override public String field() {
         return field;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return field + (lowerIncl ? "[" : "(") + lower + "; " + upper + (upperIncl ? "]" : ")");
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java
index 902db39..9872eed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.cache.query.index;
 
-import java.util.List;
+import java.util.LinkedHashMap;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
 
 /**
@@ -30,7 +30,7 @@ public interface IndexDefinition {
     public IndexName idxName();
 
     /**
-     * @return List of index key definitions.
+     * @return Ordered map of index field names to index key definitions.
      */
-    public List<IndexKeyDefinition> indexKeyDefinitions();
+    public LinkedHashMap<String, IndexKeyDefinition> indexKeyDefinitions();
 }
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 5510086..bd9496b 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
@@ -17,16 +17,13 @@
 
 package org.apache.ignite.internal.cache.query.index;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.PriorityQueue;
-import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.ignite.IgniteCheckedException;
@@ -42,7 +39,7 @@ 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.inline.IndexQueryContext;
-import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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.CacheObjectContext;
@@ -54,6 +51,7 @@ 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;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
@@ -82,10 +80,7 @@ public class IndexQueryProcessor {
     ) throws IgniteCheckedException {
         Index idx = index(cctx, idxQryDesc);
 
-        List<IndexQueryCriterion> criteria = F.isEmpty(idxQryDesc.criteria()) ?
-            Collections.emptyList() : alignCriteriaWithIndex(idxProc.indexDefinition(idx.id()), idxQryDesc);
-
-        GridCursor<IndexRow> cursor = query(cctx, idx, criteria, qryCtx);
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc, qryCtx);
 
         // Map IndexRow to Cache Key-Value pair.
         return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
@@ -128,40 +123,62 @@ public class IndexQueryProcessor {
         };
     }
 
-    /** Get index to run query by specified description. */
+    /**
+     * Finds index to run query and validates that criteria fields match a prefix of fields.
+     *
+     * @return Index to run query by specified description.
+     * @throws IgniteCheckedException If index not found.
+     */
     private Index index(GridCacheContext<?, ?> cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
-        String tableName = cctx.kernalContext().query().tableName(cctx.name(), idxQryDesc.valType());
+        final String tableName = cctx.kernalContext().query().tableName(cctx.name(), idxQryDesc.valType());
 
         if (tableName == null)
             throw failIndexQuery("No table found for type: " + idxQryDesc.valType(), null, idxQryDesc);
 
-        if (idxQryDesc.idxName() == null && !F.isEmpty(idxQryDesc.criteria())) {
-            Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
-
-            if (idx == null)
-                throw failIndexQuery("No index found for criteria", null, idxQryDesc);
-
-            return idx;
+        // Collect both fields (original and normalized).
+        Map<String, String> critFlds;
+
+        if (!F.isEmpty(idxQryDesc.criteria())) {
+            critFlds = idxQryDesc.criteria().stream()
+                .map(IndexQueryCriterion::field)
+                .flatMap(f -> {
+                    String norm = QueryUtils.normalizeObjectName(f, false);
+
+                    if (f.equals(norm))
+                        return Stream.of(new T2<>(f, f));
+                    else
+                        return Stream.of(new T2<>(f, norm), new T2<>(norm, f));
+                })
+                .collect(Collectors.toMap(IgniteBiTuple::get1, IgniteBiTuple::get2, (l, r) -> l));
         }
+        else
+            critFlds = Collections.emptyMap();
+
+        if (idxQryDesc.idxName() == null && !critFlds.isEmpty())
+            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();
 
-        Index idx = indexByName(cctx, idxName, tableName);
-
-        if (idx == null)
-            throw failIndexQuery("No index found for name: " + idxName, null, idxQryDesc);
-
-        return idx;
+        return indexByName(cctx, idxName, tableName, idxQryDesc, critFlds);
     }
 
-    /** Get index by name, or return {@code null}. */
-    private Index indexByName(GridCacheContext<?, ?> cctx, String idxName, String tableName) {
+    /**
+     * @return Index found by name.
+     * @throws IgniteCheckedException If index not found.
+     */
+    private Index indexByName(
+        GridCacheContext<?, ?> cctx,
+        String idxName,
+        String tableName,
+        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 = idxProc.index(name);
+        Index idx = getAndValidateIndex(name, idxQryDesc, criteriaFlds, false);
 
         if (idx != null)
             return idx;
@@ -171,37 +188,52 @@ public class IndexQueryProcessor {
         if (!QueryUtils.PRIMARY_KEY_INDEX.equals(idxName))
             normIdxName = QueryUtils.normalizeObjectName(idxName, false);
 
-        if (normIdxName.equals(idxName))
-            return null;
-
         name = new IndexName(cctx.name(), schema, tableName, normIdxName);
 
-        return idxProc.index(name);
+        return getAndValidateIndex(name, idxQryDesc, criteriaFlds, true);
+    }
+
+    /** */
+    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))
+            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;
     }
 
     /**
-     * Get index by list of fields to query, or return {@code null}.
+     * @return Index found by list of criteria fields.
+     * @throws IgniteCheckedException if suitable index not found.
      */
-    private Index indexByCriteria(GridCacheContext<?, ?> cctx, IndexQueryDesc idxQryDesc, String tableName) {
+    private Index indexByCriteria(
+        GridCacheContext<?, ?> cctx,
+        final Map<String, String> criteriaFlds,
+        String tableName,
+        IndexQueryDesc idxQryDesc
+    ) throws IgniteCheckedException {
         Collection<Index> idxs = idxProc.indexes(cctx);
 
-        // Check both fields (original and normalized).
-        final Set<String> critFields = idxQryDesc.criteria().stream()
-            .map(IndexQueryCriterion::field)
-            .flatMap(f -> Stream.of(f, QueryUtils.normalizeObjectName(f, false)))
-            .collect(Collectors.toSet());
-
         for (Index idx: idxs) {
             IndexDefinition idxDef = idxProc.indexDefinition(idx.id());
 
             if (!tableName.equals(idxDef.idxName().tableName()))
                 continue;
 
-            if (checkIndex(idxDef, idxQryDesc.criteria().size(), critFields))
+            if (checkIndex(idxDef, criteriaFlds))
                 return idx;
         }
 
-        return null;
+        throw failIndexQuery("No index found for criteria", null, idxQryDesc);
     }
 
     /**
@@ -209,19 +241,19 @@ public class IndexQueryProcessor {
      *
      * Criteria fields have to match to a prefix of the index. Order of fields in criteria doesn't matter.
      */
-    private boolean checkIndex(IndexDefinition idxDef, int critLen, final Set<String> criteriaFlds) {
-        if (critLen > idxDef.indexKeyDefinitions().size())
-            return false;
-
-        int matches = 0;
+    private boolean checkIndex(IndexDefinition idxDef, Map<String, String> criteriaFlds) {
+        Map<String, String> flds = new HashMap<>(criteriaFlds);
 
-        for (int i = 0; i < idxDef.indexKeyDefinitions().size(); i++) {
-            String fld = idxDef.indexKeyDefinitions().get(i).name();
+        for (String idxFldName: idxDef.indexKeyDefinitions().keySet()) {
+            String alias = flds.remove(idxFldName);
 
-            if (!criteriaFlds.contains(fld))
+            // Has not to be null, as criteriaFlds contains both original and normalized field names.
+            if (alias == null)
                 return false;
 
-            if (++matches == critLen)
+            flds.remove(alias);
+
+            if (flds.isEmpty())
                 return true;
         }
 
@@ -229,11 +261,6 @@ public class IndexQueryProcessor {
     }
 
     /** */
-    private IgniteCheckedException failIndexQueryCriteria(IndexDefinition idxDef, IndexQueryDesc idxQryDesc) {
-        return failIndexQuery( "Index doesn't match query", idxDef, idxQryDesc);
-    }
-
-    /** */
     private IgniteCheckedException failIndexQuery(String msg, IndexDefinition idxDef, IndexQueryDesc desc) {
         String exMsg = "Failed to parse IndexQuery. " + msg + ".";
 
@@ -243,128 +270,143 @@ public class IndexQueryProcessor {
         return new IgniteCheckedException(exMsg + " Query=" + desc);
     }
 
-    /** Checks that specified index matches index query criteria. */
-    private List<IndexQueryCriterion> alignCriteriaWithIndex(IndexDefinition idxDef, IndexQueryDesc idxQryDesc)
-        throws IgniteCheckedException {
-        if (idxQryDesc.criteria().size() > idxDef.indexKeyDefinitions().size())
-            throw failIndexQueryCriteria(idxDef, idxQryDesc);
+    /** Merges multiple criteria for the same field into single criterion. */
+    private Map<String, RangeIndexQueryCriterion> mergeIndexQueryCriteria(
+        InlineIndexImpl idx,
+        SortedIndexDefinition idxDef,
+        IndexQueryDesc idxQryDesc
+    ) throws IgniteCheckedException {
+        Map<String, RangeIndexQueryCriterion> mergedCriteria = new HashMap<>();
 
-        Map<String, IndexQueryCriterion> critFlds = new HashMap<>();
-        Map<String, IndexQueryCriterion> normCritFlds = new HashMap<>();
+        Map<String, IndexKeyDefinition> idxFlds = idxDef.indexKeyDefinitions();
+        IndexKeyTypeSettings keyTypeSettings = idx.segment(0).rowHandler().indexKeyTypeSettings();
+        CacheObjectContext coctx = idx.segment(0).cacheGroupContext().cacheObjectContext();
 
-        // We need check both original and normalized field names.
-        idxQryDesc.criteria().forEach(c -> {
-            critFlds.put(c.field(), c);
-            normCritFlds.put(c.field().toUpperCase(), c);
-        });
+        IndexRowComparator keyCmp = idxDef.rowComparator();
 
-        List<IndexQueryCriterion> aligned = new ArrayList<>();
+        for (IndexQueryCriterion c: idxQryDesc.criteria()) {
+            RangeIndexQueryCriterion crit = (RangeIndexQueryCriterion) c;
 
-        // Checks that users criteria matches a prefix subset of index fields.
-        for (int i = 0; i < idxQryDesc.criteria().size(); i++) {
-            String idxFld = idxDef.indexKeyDefinitions().get(i).name();
+            String fldName = idxFlds.containsKey(crit.field()) ? crit.field()
+                : QueryUtils.normalizeObjectName(crit.field(), false);
+
+            IndexKeyDefinition keyDef = idxFlds.get(fldName);
 
-            IndexQueryCriterion c = normCritFlds.remove(idxFld);
+            if (keyDef == null)
+                throw failIndexQuery("Index doesn't match criteria", idxDef, idxQryDesc);
 
-            if (c == null) {
-                // Check this field is escaped.
-                c = critFlds.remove(idxFld);
+            IndexKey l = key(crit.lower(), crit.lowerNull(), keyDef, keyTypeSettings, coctx);
+            IndexKey u = key(crit.upper(), crit.upperNull(), keyDef, keyTypeSettings, coctx);
 
-                if (c == null)
-                    throw failIndexQueryCriteria(idxDef, idxQryDesc);
+            if (l != null && u != null && keyCmp.compareKey(l, u) > 0) {
+                throw failIndexQuery("Illegal criterion: lower boundary is greater than the upper boundary: " +
+                    rangeDesc(crit, fldName, null, null), idxDef, idxQryDesc);
             }
-            else
-                critFlds.remove(c.field());
 
-            aligned.add(c);
-        }
+            boolean lowIncl = crit.lowerIncl();
+            boolean upIncl = crit.upperIncl();
 
-        if (!critFlds.isEmpty())
-            throw failIndexQueryCriteria(idxDef, idxQryDesc);
+            boolean lowNull = crit.lowerNull();
+            boolean upNull = crit.upperNull();
 
-        return aligned;
-    }
+            if (mergedCriteria.containsKey(fldName)) {
+                RangeIndexQueryCriterion prev = mergedCriteria.get(fldName);
 
-    /**
-     * Runs an index query.
-     *
-     * @return Result cursor over index segments.
-     */
-    private GridCursor<IndexRow> query(GridCacheContext<?, ?> cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
-        throws IgniteCheckedException {
+                IndexKey prevLower = (IndexKey)prev.lower();
+                IndexKey prevUpper = (IndexKey)prev.upper();
 
-        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+                // Validate merged criteria.
+                if (!checkBoundaries(l, prevUpper, crit.lowerIncl(), prev.upperIncl(), keyCmp) ||
+                    !checkBoundaries(prevLower, u, prev.lowerIncl(), crit.upperIncl(), keyCmp)) {
 
-        if (segmentsCnt == 1)
-            return query(0, idx, criteria, qryCtx);
+                    String prevDesc = rangeDesc(prev, null,
+                        prevLower == null ? null : prevLower.key(),
+                        prevUpper == null ? null : prevUpper.key());
 
-        final GridCursor<IndexRow>[] segmentCursors = new GridCursor[segmentsCnt];
+                    throw failIndexQuery("Failed to merge criterion " + rangeDesc(crit, fldName, null, null) +
+                        " with previous criteria range " + prevDesc, idxDef, idxQryDesc);
+                }
 
-        // Actually it just traverses BPlusTree to find boundaries. It's too fast to parallelize this.
-        for (int i = 0; i < segmentsCnt; i++)
-            segmentCursors[i] = query(i, idx, criteria, qryCtx);
+                int lowCmp = 0;
 
-        return new SegmentedIndexCursor(
-            segmentCursors, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
-    }
+                // Use previous lower boudary, as it's greater than the current.
+                if (l == null || (prevLower != null && (lowCmp = keyCmp.compareKey(prevLower, l)) >= 0)) {
+                    l = prevLower;
+                    lowIncl = lowCmp != 0 ? prev.lowerIncl() : prev.lowerIncl() ? lowIncl : prev.lowerIncl();
+                    lowNull = prev.lowerNull();
+                }
 
-    /**
-     * Runs an index query for single {@code segment}.
-     *
-     * @return Result cursor over segment.
-     */
-    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
-        throws IgniteCheckedException {
+                int upCmp = 0;
+
+                // Use previous upper boudary, as it's less than the current.
+                if (u == null || (prevUpper != null && (upCmp = keyCmp.compareKey(prevUpper, u)) <= 0)) {
+                    u = prevUpper;
+                    upIncl = upCmp != 0 ? prev.upperIncl() : prev.upperIncl() ? upIncl : prev.upperIncl();
+                    upNull = prev.upperNull();
+                }
+            }
 
-        if (F.isEmpty(criteria) || criteria.get(0) instanceof RangeIndexQueryCriterion)
-            return treeIndexRange((InlineIndex) idx, criteria, segment, qryCtx);
+            RangeIndexQueryCriterion idxKeyCrit = new RangeIndexQueryCriterion(fldName, l, u);
+            idxKeyCrit.lowerIncl(lowIncl);
+            idxKeyCrit.upperIncl(upIncl);
+            idxKeyCrit.lowerNull(lowNull);
+            idxKeyCrit.upperNull(upNull);
 
-        throw new IllegalStateException("Doesn't support index query criteria: " + criteria.getClass().getName());
+            mergedCriteria.put(fldName, idxKeyCrit);
+        }
+
+        return mergedCriteria;
     }
 
     /**
-     * Runs range query over specified segment. There are 2 steps to run query:
-     * 1. Traverse index by specified boundaries;
-     * 2. Scan over cursor and filter rows that doesn't match user criteria.
-     *
-     * Filtering is required in 2 cases:
-     * 1. Exclusion of one of boundaries, as idx.find() includes both of them;
-     * 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.
+     * @return {@code} true if boudaries are intersected, otherwise {@code false}.
      */
-    private GridCursor<IndexRow> treeIndexRange(InlineIndex idx, List<IndexQueryCriterion> criteria, int segment,
-        IndexQueryContext qryCtx) throws IgniteCheckedException {
+    private boolean checkBoundaries(
+        IndexKey left,
+        IndexKey right,
+        boolean leftIncl,
+        boolean rightIncl,
+        IndexRowComparator keyCmp
+    ) throws IgniteCheckedException {
+        boolean boundaryCheck = left != null && right != null;
 
-        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
-        CacheObjectContext coctx = idx.segment(0).cacheGroupContext().cacheObjectContext();
+        if (boundaryCheck) {
+            int cmp = keyCmp.compareKey(left, right);
+
+            return cmp < 0 || (cmp == 0 && leftIncl && rightIncl);
+        }
+
+        return true;
+    }
 
-        IndexKey[] lowerBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
-        IndexKey[] upperBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+    /** Checks that specified index matches index query criteria. */
+    private IndexRangeQuery alignCriteriaWithIndex(
+        InlineIndexImpl idx,
+        Map<String, RangeIndexQueryCriterion> criteria,
+        IndexDefinition idxDef
+    ) {
+        // Size of bounds array has to be equal to count of indexed fields.
+        IndexKey[] lowerBounds = new IndexKey[idxDef.indexKeyDefinitions().size()];
+        IndexKey[] upperBounds = new IndexKey[idxDef.indexKeyDefinitions().size()];
 
         boolean lowerAllNulls = true;
         boolean upperAllNulls = true;
 
-        List<RangeIndexQueryCriterion> treeCriteria = new ArrayList<>();
-
-        for (int i = 0; i < criteria.size(); i++) {
-            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+        IndexRangeQuery qry = new IndexRangeQuery(criteria.size());
 
-            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+        // Checks that users criteria matches a prefix subset of index fields.
+        int i = 0;
 
-            if (!def.name().equalsIgnoreCase(c.field()))
-                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+        for (Map.Entry<String, IndexKeyDefinition> keyDef: idxDef.indexKeyDefinitions().entrySet()) {
+            RangeIndexQueryCriterion criterion = criteria.remove(keyDef.getKey());
 
-            // If index is desc, then we need to swap boundaries as user declare criteria in straight manner.
-            // For example, there is an idx (int Val desc). It means that index stores data in reverse order (1 < 0).
-            // But user won't expect for criterion gt(1) to get 0 as result, instead user will use lt(1) for getting
-            // 0. Then we need to swap user criterion.
-            if (def.order().sortOrder() == DESC)
-                c = c.swap();
+            if (keyDef.getValue().order().sortOrder() == DESC)
+                criterion = criterion.swap();
 
-            treeCriteria.add(c);
+            qry.criteria[i] = criterion;
 
-            IndexKey l = key(c.lower(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
-            IndexKey u = key(c.upper(), c.upperNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey l = (IndexKey) criterion.lower();
+            IndexKey u = (IndexKey) criterion.upper();
 
             if (l != null)
                 lowerAllNulls = false;
@@ -373,14 +415,87 @@ public class IndexQueryProcessor {
                 upperAllNulls = false;
 
             lowerBounds[i] = l;
-            upperBounds[i] = u;
+            upperBounds[i++] = u;
+
+            if (criteria.isEmpty())
+                break;
+        }
+
+        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
+
+        qry.lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
+        qry.upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+
+        return qry;
+    }
+
+    /**
+     * Runs an index query.
+     *
+     * @return Result cursor over index segments.
+     */
+    private GridCursor<IndexRow> query(GridCacheContext<?, ?> cctx, Index idx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        IndexQueryCriterion c = F.isEmpty(idxQryDesc.criteria()) ? null : idxQryDesc.criteria().get(0);
+
+        if (c == null || c instanceof RangeIndexQueryCriterion)
+            return querySortedIndex(cctx, (InlineIndexImpl) idx, idxQryDesc, qryCtx);
+
+        throw new IllegalStateException("Doesn't support index query criteria: " + c.getClass().getName());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    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);
 
-        IndexRow lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
-        IndexRow upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+
+        if (segmentsCnt == 1)
+            return treeIndexRange(idx, 0, qry, qryCtx);
+
+        final GridCursor<IndexRow>[] segmentCursors = new GridCursor[segmentsCnt];
+
+        // Actually it just traverses BPlusTree to find boundaries. It's too fast to parallelize this.
+        for (int i = 0; i < segmentsCnt; i++)
+            segmentCursors[i] = treeIndexRange(idx, i, qry, qryCtx);
+
+        return new SegmentedIndexCursor(
+            segmentCursors, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+    }
+
+    /**
+     * Runs range query over specified segment. There are 2 steps to run query:
+     * 1. Traverse index by specified boundaries;
+     * 2. Scan over cursor and filter rows that doesn't match user criteria.
+     *
+     * Filtering is required in 2 cases:
+     * 1. Exclusion of one of boundaries, as idx.find() includes both of them;
+     * 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)
+        throws IgniteCheckedException {
+
+        InlineIndexRowHandler hnd = idx.segment(segment).rowHandler();
 
         // Step 1. Traverse index.
-        GridCursor<IndexRow> findRes = idx.find(lower, upper, segment, qryCtx);
+        GridCursor<IndexRow> findRes = idx.find(qry.lower, qry.upper, segment, qryCtx);
 
         // Step 2. Scan and filter.
         return new GridCursor<IndexRow>() {
@@ -392,7 +507,7 @@ public class IndexQueryProcessor {
                 if (!findRes.next())
                     return false;
 
-                while (rowIsOutOfRange(get(), lower, upper)) {
+                while (rowIsOutOfRange(get(), qry.lower, qry.upper)) {
                     if (!findRes.next())
                         return false;
                 }
@@ -414,15 +529,15 @@ public class IndexQueryProcessor {
                 if (low == null && high == null)
                     return false;  // Unbounded search, include all.
 
-                int criteriaKeysCnt = treeCriteria.size();
+                int criteriaKeysCnt = qry.criteria.length;
 
                 for (int i = 0; i < criteriaKeysCnt; i++) {
-                    RangeIndexQueryCriterion c = treeCriteria.get(i);
+                    RangeIndexQueryCriterion c = qry.criteria[i];
 
                     boolean descOrder = hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC;
 
                     if (low != null && low.key(i) != null) {
-                        int cmp = rowCmp.compareKey(row, low, i);
+                        int cmp = rowCmp.compareRow(row, low, i);
 
                         if (cmp == 0) {
                             if (!c.lowerIncl())
@@ -433,7 +548,7 @@ public class IndexQueryProcessor {
                     }
 
                     if (high != null && high.key(i) != null) {
-                        int cmp = rowCmp.compareKey(row, high, i);
+                        int cmp = rowCmp.compareRow(row, high, i);
 
                         if (cmp == 0) {
                             if (!c.upperIncl())
@@ -477,7 +592,7 @@ public class IndexQueryProcessor {
             cursorComp = new Comparator<GridCursor<IndexRow>>() {
                 @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
                     try {
-                        return rowCmp.compareKey(o1.get(), o2.get(), 0);
+                        return rowCmp.compareRow(o1.get(), o2.get(), 0);
                     }
                     catch (IgniteCheckedException e) {
                         throw new IgniteException(e);
@@ -513,4 +628,38 @@ public class IndexQueryProcessor {
             return head;
         }
     }
+
+    /**
+     * @return Modified description for criterion in case of error.
+     */
+    private static String rangeDesc(RangeIndexQueryCriterion c, String fldName, Object lower, Object upper) {
+        String fld = fldName == null ? c.field() : fldName;
+
+        Object l = lower == null ? c.lower() : lower;
+        Object u = upper == null ? c.upper() : upper;
+
+        RangeIndexQueryCriterion r = new RangeIndexQueryCriterion(fld, l, u);
+
+        r.lowerIncl(c.lowerIncl());
+        r.upperIncl(c.upperIncl());
+
+        return r.toString();
+    }
+
+    /** */
+    private static class IndexRangeQuery {
+        /** Ordered list of criteria. Order matches index fields order. */
+        private final RangeIndexQueryCriterion[] criteria;
+
+        /** */
+        private IndexRangeQuery(int critSize) {
+            criteria = new RangeIndexQueryCriterion[critSize];
+        }
+
+        /** */
+        private IndexRow lower;
+
+        /** */
+        private IndexRow upper;
+    }
 }
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 140f0ac..c0e51db 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
@@ -25,9 +25,6 @@ import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
  * Defines a signle index key.
  */
 public class IndexKeyDefinition {
-    /** Index key name. */
-    private final String name;
-
     /** Index key type. {@link IndexKeyTypes}. */
     private final int idxType;
 
@@ -38,10 +35,9 @@ public class IndexKeyDefinition {
     private final int precision;
 
     /** */
-    public IndexKeyDefinition(String name, int idxType, Order order, long precision) {
+    public IndexKeyDefinition(int idxType, Order order, long precision) {
         this.idxType = idxType;
         this.order = order;
-        this.name = name;
 
         // Workaround due to wrong type conversion (int -> long).
         if (precision >= Integer.MAX_VALUE)
@@ -61,11 +57,6 @@ public class IndexKeyDefinition {
     }
 
     /** */
-    public String name() {
-        return name;
-    }
-
-    /** */
     public int precision() {
         return precision;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
index c388c41..25258a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
@@ -31,16 +31,24 @@ public interface IndexRowComparator {
      * @param off offset of an index key.
      * @param maxSize max size to read.
      * @param key key to compare with.
-     * @param curType type of an index key.
+     * @param curType type of index key.
      */
     public int compareKey(long pageAddr, int off, int maxSize, IndexKey key, int curType) throws IgniteCheckedException;
 
     /**
      * Compare index keys.
      *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareKey(IndexKey left, IndexKey right) throws IgniteCheckedException;
+
+    /**
+     * Compare index rows by key specified with {@code idx}.
+     *
      * @param left index row.
      * @param right index row.
      * @param idx offset of index key.
      */
-    public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+    public int compareRow(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
 }
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 3b2b919..916fe25 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
@@ -54,10 +54,20 @@ public class IndexRowCompartorImpl implements IndexRowComparator {
     }
 
     /** {@inheritDoc} */
-    @Override public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException {
-        IndexKey lkey = left.key(idx);
-        IndexKey rkey = right.key(idx);
+    @Override public int compareRow(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException {
+        return compare(left.key(idx), right.key(idx));
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareKey(IndexKey left, IndexKey right) throws IgniteCheckedException {
+        if (left == right)
+            return 0;
 
+        return compare(left, right);
+    }
+
+    /** */
+    private int compare(IndexKey lkey, IndexKey rkey) {
         if (lkey == NullIndexKey.INSTANCE)
             return lkey.compare(rkey);
         else if (rkey == NullIndexKey.INSTANCE)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
index d09b17c..7bf3fc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
@@ -183,7 +183,7 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
             if (!(v1 != null && v2 != null))
                 return false;
 
-            if (def.rowComparator().compareKey((IndexRow) r1, (IndexRow) r2, i) != 0)
+            if (def.rowComparator().compareRow(r1, r2, i) != 0)
                 return false;
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java
index ad99afd..7fc526c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -172,8 +173,8 @@ public class InlineIndexKeyTypeRegistry {
 
     /**
      * Return list of key types for specified key definitions and key type settings.
-     * */
-    public static List<InlineIndexKeyType> types(List<IndexKeyDefinition> keyDefs, IndexKeyTypeSettings settings) {
+     */
+    public static List<InlineIndexKeyType> types(Collection<IndexKeyDefinition> keyDefs, IndexKeyTypeSettings settings) {
         List<InlineIndexKeyType> keyTypes = new ArrayList<>();
 
         for (IndexKeyDefinition keyDef: keyDefs) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
index 480413e..19209d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
@@ -221,10 +221,10 @@ public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
             return metaInfo.inlineObjectSupported();
         else {
             try {
-                if (InlineObjectBytesDetector.objectMayBeInlined(metaInfo.inlineSize(), def.indexKeyDefinitions())) {
+                if (InlineObjectBytesDetector.objectMayBeInlined(metaInfo.inlineSize(), def.indexKeyDefinitions().values())) {
                     try {
                         InlineObjectBytesDetector inlineObjDetector = new InlineObjectBytesDetector(
-                            metaInfo.inlineSize(), def.indexKeyDefinitions(), def.idxName(), log);
+                            metaInfo.inlineSize(), def.indexKeyDefinitions().values(), def.idxName(), log);
 
                         // Create a settings for case where java objects inilned as byte array.
                         IndexKeyTypeSettings keyTypeSettings = new IndexKeyTypeSettings()
@@ -273,7 +273,7 @@ public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
 
         int off = io.offset(idx);
 
-        List<IndexKeyDefinition> keyDefs = def.indexKeyDefinitions();
+        List<IndexKeyDefinition> keyDefs = rowHnd.indexKeyDefinitions();
 
         List<InlineIndexKeyType> keyTypes = rowHandler().inlineIndexKeyTypes();
 
@@ -355,10 +355,10 @@ public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
             if (row.key(i) == null)
                 return 0;
 
-            int c = def.rowComparator().compareKey(currRow, row, i);
+            int c = def.rowComparator().compareRow(currRow, row, i);
 
             if (c != 0)
-                return applySortOrder(Integer.signum(c), def.indexKeyDefinitions().get(i).order().sortOrder());
+                return applySortOrder(Integer.signum(c), rowHnd.indexKeyDefinitions().get(i).order().sortOrder());
         }
 
         return 0;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java
index 9275749..b621799 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java
@@ -18,7 +18,8 @@
 package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
 import java.util.Arrays;
-import java.util.List;
+import java.util.Collection;
+import java.util.Iterator;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.cache.query.index.IndexName;
@@ -46,7 +47,7 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<Index
     private final int inlineSize;
 
     /** Inline helpers. */
-    private final List<IndexKeyDefinition> keyDefs;
+    private final Collection<IndexKeyDefinition> keyDefs;
 
     /** Inline object supported flag. */
     private boolean inlineObjSupported = true;
@@ -63,7 +64,7 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<Index
      * @param idxName Index name.
      * @param log Ignite logger.
      */
-    public InlineObjectBytesDetector(int inlineSize, List<IndexKeyDefinition> keyDefs, IndexName idxName,
+    public InlineObjectBytesDetector(int inlineSize, Collection<IndexKeyDefinition> keyDefs, IndexName idxName,
         IgniteLogger log) {
         this.inlineSize = inlineSize;
         this.keyDefs = keyDefs;
@@ -85,8 +86,10 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<Index
 
         IndexKeyTypeSettings keyTypeSettings = new IndexKeyTypeSettings();
 
+        Iterator<IndexKeyDefinition> it = keyDefs.iterator();
+
         for (int i = 0; i < keyDefs.size(); ++i) {
-            IndexKeyDefinition keyDef = keyDefs.get(i);
+            IndexKeyDefinition keyDef = it.next();
 
             if (fieldOff >= inlineSize)
                 return false;
@@ -172,7 +175,7 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<Index
      *
      * @return {@code true} If the object may be inlined.
      */
-    public static boolean objectMayBeInlined(int inlineSize, List<IndexKeyDefinition> keyDefs) {
+    public static boolean objectMayBeInlined(int inlineSize, Collection<IndexKeyDefinition> keyDefs) {
         int remainSize = inlineSize;
 
         // The settings does not affect on inline size.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
index 16820a7..fc60069 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
@@ -23,7 +23,6 @@ import java.util.stream.Collectors;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.SystemProperty;
-import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
 import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -105,8 +104,7 @@ public class InlineRecommender {
                     break;
             }
 
-            String cols = def.indexKeyDefinitions().stream()
-                .map(IndexKeyDefinition::name)
+            String cols = def.indexKeyDefinitions().keySet().stream()
                 .collect(Collectors.joining(", ", "(", ")"));
 
             String type = def.primary() ? "PRIMARY KEY" : def.affinity() ? "AFFINITY KEY (implicit)" : "SECONDARY";
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 9bfa47c..656f234 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -473,6 +473,22 @@ public final class GridTestUtils {
      */
     public static Throwable assertThrows(@Nullable IgniteLogger log, Callable<?> call,
         Class<? extends Throwable> cls, @Nullable String msg) {
+        return assertThrows(log, call, cls, msg, null);
+    }
+
+    /**
+     * Checks whether callable throws expected exception or not.
+     *
+     * @param log Logger (optional).
+     * @param call Callable.
+     * @param cls Exception class.
+     * @param msg Exception message (optional). If provided exception message
+     *      and this message should be equal.
+     * @param notThrowsMsg Optional exception message if expected exception wasn't thrown.
+     * @return Thrown throwable.
+     */
+    public static Throwable assertThrows(@Nullable IgniteLogger log, Callable<?> call,
+        Class<? extends Throwable> cls, @Nullable String msg, @Nullable String notThrowsMsg) {
         assert call != null;
         assert cls != null;
 
@@ -506,7 +522,9 @@ public final class GridTestUtils {
             return e;
         }
 
-        throw new AssertionError("Exception has not been thrown.");
+        String asrtMsg = notThrowsMsg == null ? "Exception has not been thrown." : notThrowsMsg;
+
+        throw new AssertionError(asrtMsg);
     }
 
     /**
diff --git a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialIndexDefinition.java b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialIndexDefinition.java
index be88f34..83c6ed5 100644
--- a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialIndexDefinition.java
+++ b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialIndexDefinition.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
-import java.util.List;
+import java.util.LinkedHashMap;
 import org.apache.ignite.internal.cache.query.index.IndexDefinition;
 import org.apache.ignite.internal.cache.query.index.IndexName;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
@@ -31,15 +31,20 @@ public class GeoSpatialIndexDefinition implements IndexDefinition {
     private final QueryIndexRowHandler rowHnd;
 
     /** */
+    private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+    /** */
     private final int segmentsCnt;
 
     /** */
     private final IndexName idxName;
 
     /** */
-    public GeoSpatialIndexDefinition(IndexName idxName, QueryIndexRowHandler rowHnd, int segmentsCnt) {
+    public GeoSpatialIndexDefinition(IndexName idxName, LinkedHashMap<String, IndexKeyDefinition> keyDefs,
+        QueryIndexRowHandler rowHnd, int segmentsCnt) {
         this.idxName = idxName;
         this.rowHnd = rowHnd;
+        this.keyDefs = keyDefs;
         this.segmentsCnt = segmentsCnt;
     }
 
@@ -59,7 +64,7 @@ public class GeoSpatialIndexDefinition implements IndexDefinition {
     }
 
     /** {@inheritDoc} */
-    @Override public List<IndexKeyDefinition> indexKeyDefinitions() {
-        return rowHnd.indexKeyDefinitions();
+    @Override public LinkedHashMap<String, IndexKeyDefinition> indexKeyDefinitions() {
+        return keyDefs;
     }
 }
diff --git a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialUtils.java b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialUtils.java
index c7c9c13..0895d6e 100644
--- a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialUtils.java
+++ b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialUtils.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
+import java.util.LinkedHashMap;
 import java.util.List;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.cache.query.index.Index;
@@ -49,15 +50,15 @@ public class GeoSpatialUtils {
         try {
             IndexName name = new IndexName(tbl.cacheName(), tbl.getSchema().getName(), tbl.getName(), idxName);
 
-            List<IndexKeyDefinition> keyDefs = new QueryIndexKeyDefinitionProvider(tbl, cols).keyDefinitions();
+            LinkedHashMap<String, IndexKeyDefinition> keyDefs = new QueryIndexKeyDefinitionProvider(tbl, cols).keyDefinitions();
 
-            List<InlineIndexKeyType> idxKeyTypes = InlineIndexKeyTypeRegistry.types(keyDefs, DUMMY_SETTINGS);
+            List<InlineIndexKeyType> idxKeyTypes = InlineIndexKeyTypeRegistry.types(keyDefs.values(), DUMMY_SETTINGS);
 
             QueryIndexRowHandler rowHnd = new QueryIndexRowHandler(tbl, cols, keyDefs, idxKeyTypes, DUMMY_SETTINGS);
 
             final int segments = tbl.rowDescriptor().cacheInfo().config().getQueryParallelism();
 
-            IndexDefinition def = new GeoSpatialIndexDefinition(name, rowHnd, segments);
+            IndexDefinition def = new GeoSpatialIndexDefinition(name, keyDefs, rowHnd, segments);
 
             Index idx = tbl.cacheContext().kernalContext().indexProcessor().createIndex(
                 tbl.cacheContext(), GeoSpatialIndexFactory.INSTANCE, def);
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 2511337..5530b85 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
@@ -87,8 +87,8 @@ public class H2RowComparator extends IndexRowCompartorImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException {
-        int cmp = super.compareKey(left, right, idx);
+    @Override public int compareRow(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException {
+        int cmp = super.compareRow(left, right, idx);
 
         if (cmp != COMPARE_UNSUPPORTED)
             return cmp;
@@ -114,6 +114,21 @@ public class H2RowComparator extends IndexRowCompartorImpl {
         return Integer.signum(c);
     }
 
+    /** {@inheritDoc} */
+    @Override public int compareKey(IndexKey left, IndexKey right) throws IgniteCheckedException {
+        int cmp = super.compareKey(left, right);
+
+        if (cmp != COMPARE_UNSUPPORTED)
+            return cmp;
+
+        int ltype = DataType.getTypeFromClass(left.key().getClass());
+        int rtype = DataType.getTypeFromClass(right.key().getClass());
+
+        int c = compareValues(wrap(left.key(), ltype), wrap(right.key(), rtype));
+
+        return Integer.signum(c);
+    }
+
     /** */
     private Value wrap(Object val, int type) throws IgniteCheckedException {
         return H2Utils.wrap(coctx, val, type);
@@ -126,7 +141,7 @@ public class H2RowComparator extends IndexRowCompartorImpl {
      */
     public int compareValues(Value v1, Value v2) throws IgniteCheckedException {
         try {
-            return v1 == v2 ? 0 : table.compareTypeSafe(v1, v2);
+            return table.compareTypeSafe(v1, v2);
 
         } catch (DbException ex) {
             throw new IgniteCheckedException("Rows cannot be compared", ex);
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexDefinition.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexDefinition.java
index f2e9842..4492bdc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexDefinition.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexDefinition.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.index;
 
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.cache.query.index.IndexName;
@@ -40,7 +41,7 @@ import org.h2.value.CompareMode;
  */
 public class QueryIndexDefinition implements SortedIndexDefinition {
     /** Wrapped key definitions. */
-    private List<IndexKeyDefinition> keyDefs;
+    private LinkedHashMap<String, IndexKeyDefinition> keyDefs;
 
     /** List of unwrapped index columns. */
     private List<IndexColumn> h2UnwrappedCols;
@@ -126,7 +127,7 @@ public class QueryIndexDefinition implements SortedIndexDefinition {
     }
 
     /** {@inheritDoc} */
-    @Override public List<IndexKeyDefinition> indexKeyDefinitions() {
+    @Override public LinkedHashMap<String, IndexKeyDefinition> indexKeyDefinitions() {
         if (keyDefs == null)
             throw new IllegalStateException("Index key definitions is not initialized yet.");
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexKeyDefinitionProvider.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexKeyDefinitionProvider.java
index 6e07e00..73be240 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexKeyDefinitionProvider.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexKeyDefinitionProvider.java
@@ -17,8 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.index;
 
-import java.util.ArrayList;
-import java.util.Collections;
+import java.util.LinkedHashMap;
 import java.util.List;
 import org.apache.ignite.internal.cache.query.index.NullsOrder;
 import org.apache.ignite.internal.cache.query.index.Order;
@@ -35,8 +34,8 @@ public class QueryIndexKeyDefinitionProvider {
     /** H2 index columns. */
     private final List<IndexColumn> h2IdxColumns;
 
-    /** Unmodified list of index key definitions. */
-    private List<IndexKeyDefinition> keyDefs;
+    /** Unmodifiable ordered map of index key definitions. */
+    private LinkedHashMap<String, IndexKeyDefinition> keyDefs;
 
     /** */
     public QueryIndexKeyDefinitionProvider(GridH2Table table, List<IndexColumn> h2IdxColumns) {
@@ -47,26 +46,25 @@ public class QueryIndexKeyDefinitionProvider {
     /**
      * @return List of index key definitions.
      */
-    public List<IndexKeyDefinition> keyDefinitions() {
+    public LinkedHashMap<String, IndexKeyDefinition> keyDefinitions() {
         if (keyDefs != null)
             return keyDefs;
 
-        List<IndexKeyDefinition> idxKeyDefinitions = new ArrayList<>();
+        LinkedHashMap<String, IndexKeyDefinition> idxKeyDefinitions = new LinkedHashMap<>();
 
         for (IndexColumn c: h2IdxColumns)
-            idxKeyDefinitions.add(keyDefinition(c));
+            idxKeyDefinitions.put(c.columnName, keyDefinition(c));
 
         IndexColumn.mapColumns(h2IdxColumns.toArray(new IndexColumn[0]), table);
 
-        keyDefs = Collections.unmodifiableList(idxKeyDefinitions);
+        keyDefs = idxKeyDefinitions;
 
         return keyDefs;
     }
 
     /** */
     private IndexKeyDefinition keyDefinition(IndexColumn c) {
-        return new IndexKeyDefinition(
-            c.columnName, c.column.getType(), sortOrder(c.sortType), c.column.getPrecision());
+        return new IndexKeyDefinition(c.column.getType(), sortOrder(c.sortType), c.column.getPrecision());
     }
 
     /** Maps H2 column order to Ignite index order. */
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexRowHandler.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexRowHandler.java
index 9fcfb40..5814a33 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexRowHandler.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexRowHandler.java
@@ -17,6 +17,9 @@
 
 package org.apache.ignite.internal.processors.query.h2.index;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
 import java.util.List;
 import org.apache.ignite.internal.binary.BinaryObjectImpl;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
@@ -56,10 +59,10 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
 
     /** */
     public QueryIndexRowHandler(GridH2Table h2table, List<IndexColumn> h2IdxColumns,
-        List<IndexKeyDefinition> keyDefs, List<InlineIndexKeyType> keyTypes, IndexKeyTypeSettings keyTypeSettings) {
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs, List<InlineIndexKeyType> keyTypes, IndexKeyTypeSettings keyTypeSettings) {
         this.h2IdxColumns = h2IdxColumns;
         this.keyTypes = keyTypes;
-        this.keyDefs = keyDefs;
+        this.keyDefs = Collections.unmodifiableList(new ArrayList<>(keyDefs.values()));
         this.h2Table = h2table;
         cacheDesc = h2table.rowDescriptor();
         this.keyTypeSettings = keyTypeSettings;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryRowHandlerFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryRowHandlerFactory.java
index 6d1eddb..b9d6cbd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryRowHandlerFactory.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryRowHandlerFactory.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.index;
 
+import java.util.LinkedHashMap;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
@@ -38,10 +39,10 @@ public class QueryRowHandlerFactory implements InlineIndexRowHandlerFactory {
 
         QueryIndexDefinition def = (QueryIndexDefinition) sdef;
 
-        List<IndexKeyDefinition> keyDefs = def.indexKeyDefinitions();
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs = def.indexKeyDefinitions();
         List<IndexColumn> h2IdxColumns = def.getColumns();
 
-        List<InlineIndexKeyType> keyTypes = InlineIndexKeyTypeRegistry.types(keyDefs, keyTypeSettings);
+        List<InlineIndexKeyType> keyTypes = InlineIndexKeyTypeRegistry.types(keyDefs.values(), keyTypeSettings);
 
         return new QueryIndexRowHandler(def.getTable(), h2IdxColumns, keyDefs, keyTypes, keyTypeSettings);
     }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexDefinition.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexDefinition.java
index 536d100..6e95f37 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexDefinition.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexDefinition.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2.index.client;
 
+import java.util.LinkedHashMap;
 import java.util.List;
 import org.apache.ignite.internal.cache.query.index.IndexDefinition;
 import org.apache.ignite.internal.cache.query.index.IndexName;
@@ -39,7 +40,7 @@ public class ClientIndexDefinition implements IndexDefinition {
     private final IndexName idxName;
 
     /** */
-    private final List<IndexKeyDefinition> keyDefs;
+    private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
 
     /** */
     public ClientIndexDefinition(GridH2Table table, IndexName idxName, List<IndexColumn> unwrappedCols,
@@ -67,7 +68,7 @@ public class ClientIndexDefinition implements IndexDefinition {
     }
 
     /** {@inheritDoc} */
-    @Override public List<IndexKeyDefinition> indexKeyDefinitions() {
+    @Override public LinkedHashMap<String, IndexKeyDefinition> indexKeyDefinitions() {
         return keyDefs;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexFactory.java
index 7092e3e..50372bc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexFactory.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientIndexFactory.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.query.h2.index.client;
 
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
 import java.util.List;
 import org.apache.ignite.internal.cache.query.index.Index;
 import org.apache.ignite.internal.cache.query.index.IndexDefinition;
@@ -45,11 +47,12 @@ public class ClientIndexFactory implements IndexFactory {
     @Override public Index createIndex(GridCacheContext<?, ?> cctx, IndexDefinition definition) {
         ClientIndexDefinition def = (ClientIndexDefinition) definition;
 
-        List<IndexKeyDefinition> keyDefs = definition.indexKeyDefinitions();
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs = definition.indexKeyDefinitions();
 
-        List<InlineIndexKeyType> keyTypes = InlineIndexKeyTypeRegistry.types(keyDefs, DUMMY_SETTINGS);
+        List<InlineIndexKeyType> keyTypes = InlineIndexKeyTypeRegistry.types(keyDefs.values(), DUMMY_SETTINGS);
 
-        int inlineSize = InlineIndexTree.computeInlineSize(keyTypes, keyDefs, def.getCfgInlineSize(), def.getMaxInlineSize());
+        int inlineSize = InlineIndexTree.computeInlineSize(keyTypes, new ArrayList<>(keyDefs.values()),
+            def.getCfgInlineSize(), def.getMaxInlineSize());
 
         return new ClientInlineIndex(def.idxName().idxName(), inlineSize);
     }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index b98bb2c..1f84256 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.opt;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -748,7 +749,7 @@ public class GridH2Table extends TableBase {
                 }
 
                 /** {@inheritDoc} */
-                @Override public List<IndexKeyDefinition> indexKeyDefinitions() {
+                @Override public LinkedHashMap<String, IndexKeyDefinition> indexKeyDefinitions() {
                     throw new UnsupportedOperationException("Hasn't be invoked for destroyed index.");
                 }
             };
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFailoverTest.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFailoverTest.java
index 1461a22..a95b24d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFailoverTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFailoverTest.java
@@ -17,9 +17,11 @@
 
 package org.apache.ignite.cache.query;
 
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
+import java.util.stream.Stream;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -41,6 +43,8 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+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.lt;
 
 /** */
@@ -104,14 +108,6 @@ public class IndexQueryFailoverTest extends GridCommonAbstractTest {
                 return cache.query(qryNullCriteria);
             },
             NullPointerException.class, "Ouch! Argument cannot be null: field");
-
-        GridTestUtils.assertThrowsAnyCause(null, () -> {
-                IndexQuery<Long, Person> qryDuplicateField = new IndexQuery<Long, Person>(Person.class, qryIdx)
-                    .setCriteria(lt("id", 12), lt("id", 32));
-
-                return cache.query(qryDuplicateField);
-            },
-            IllegalArgumentException.class, "Ouch! Argument is invalid");
     }
 
     /** */
@@ -155,7 +151,7 @@ public class IndexQueryFailoverTest extends GridCommonAbstractTest {
     /** */
     @Test
     public void testQueryWrongQuery() {
-        String errMsg = qryIdx != null ? "Index doesn't match query." : "No index found for criteria.";
+        String errMsg = qryIdx != null ? "Index doesn't match criteria." : "No index found for criteria.";
 
         GridTestUtils.assertThrowsAnyCause(null, () -> {
                 IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
@@ -174,11 +170,38 @@ public class IndexQueryFailoverTest extends GridCommonAbstractTest {
                 return cache.query(qry).getAll();
             },
             IgniteCheckedException.class, errMsg);
+
+        GridTestUtils.assertThrowsAnyCause(null, () -> {
+                IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
+                    .setCriteria(between("id", 432, 40));
+
+                return cache.query(qry).getAll();
+            },
+            IgniteCheckedException.class, "Illegal criterion: lower boundary is greater than the upper boundary: " +
+                "ID[432; 40]");
+
+        Stream.of(
+            Arrays.asList(lt("id", 100), gt("id", 101)),
+            Arrays.asList(eq("id", 100), eq("id", 101)),
+            Arrays.asList(eq("id", 101), eq("id", 100)),
+            Arrays.asList(eq("id", 101), between("id", 19, 40))
+        ).forEach(crit -> {
+            String msg = "Failed to merge criterion " + crit.get(1).toString().replace("id", "ID")
+                + " with previous criteria range " + crit.get(0).toString().replace("id", "ID");
+
+            GridTestUtils.assertThrowsAnyCause(null, () -> {
+                    IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
+                        .setCriteria(crit);
+
+                    return cache.query(qry).getAll();
+                },
+                IgniteCheckedException.class, msg);
+        });
     }
 
     /** */
     @Test
-    public void testRangeQueries() {
+    public void testStopNode() {
         insertData(0, CNT);
 
         IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, qryIdx)
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 bc9dfd7..1825f86 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
@@ -125,7 +125,7 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
 
                 return tblCache.query(wrongQry).getAll();
 
-            }, IgniteCheckedException.class, "Index doesn't match query.");
+            }, IgniteCheckedException.class, "Index doesn't match criteria.");
         }
 
         // Wrong cache.
@@ -176,7 +176,7 @@ public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
 
         check(tblCache.query(qry), 0, pivot);
 
-        String errMsg = qryDescIdxName != null ? "Index doesn't match query." : "No index found for criteria.";
+        String errMsg = qryDescIdxName != null ? "Index doesn't match criteria." : "No index found for criteria.";
 
         GridTestUtils.assertThrowsAnyCause(null, () -> {
             IndexQuery<Long, Object> wrongQry = new IndexQuery<Long, Object>(Person.class.getName(), qryDescIdxName)
diff --git a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryTestSuite.java
index 4237692..e141e27 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryTestSuite.java
@@ -36,7 +36,8 @@ import org.junit.runners.Suite;
     IndexQueryRangeTest.class,
     IndexQueryWrongIndexTest.class,
     MultifieldIndexQueryTest.class,
-    MultiTableIndexQuery.class
+    MultiTableIndexQuery.class,
+    RepeatedFieldIndexQueryTest.class
 })
 public class IndexQueryTestSuite {
 }
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
new file mode 100644
index 0000000..fff71a2
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
@@ -0,0 +1,360 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+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.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+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;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<IndexQueryCriterion> criteria = criteria(fldName, lower, upper);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> checks = new ArrayList<>();
+
+        for (int i = 0; i < criteria.size(); i++) {
+            for (int j = 0; j < criteria.size(); j++) {
+                checks.add(new T2<>(
+                    (RangeIndexQueryCriterion)criteria.get(i),
+                    (RangeIndexQueryCriterion)criteria.get(j)));
+            }
+
+        }
+
+        checks.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMergeMultipleCriteriaForSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(gt(fldName, lower), gt(fldName, lower - 1), gt(fldName, lower - 2),
+                lt(fldName, upper), lt(fldName, upper + 1), lt(fldName, upper + 2));
+
+        check(null, cache.query(qry), lower + 1, upper);
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        checkEqualsCriteria(lower, upper, 0, CNT);
+        checkEqualsCriteria(lower, upper, CNT, 0);
+        checkEqualsCriteria(upper, lower, 0, CNT);
+        checkEqualsCriteria(upper, lower, CNT, 0);
+    }
+
+    /** */
+    private void checkEqualsCriteria(int eq1, int eq2, int from, int to) {
+        GridTestUtils.assertThrows(null, () -> {
+            IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+                .setCriteria(eq(fldName, eq1), eq(fldName, eq2), between(fldName, from, to));
+
+            return cache.query(qry).getAll();
+        }, CacheException.class, "Failed to merge criterion");
+    }
+
+    /** */
+    @Test
+    public void testCommonBoundary() {
+        int boundary = new Random().nextInt(CNT / 2);
+
+        checkEmptyForCommonBoundary(lt(fldName, boundary), gt(fldName, boundary));
+        checkEmptyForCommonBoundary(lte(fldName, boundary), gt(fldName, boundary));
+        checkEmptyForCommonBoundary(lt(fldName, boundary), gte(fldName, boundary));
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(lte(fldName, boundary), gte(fldName, boundary));
+
+        check(null, cache.query(qry), boundary, boundary + 1);
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(between(fldName, 0, boundary), between(fldName, boundary, CNT));
+
+        check(null, cache.query(qry), boundary, boundary + 1);
+    }
+
+    /** */
+    private void checkEmptyForCommonBoundary(IndexQueryCriterion c1, IndexQueryCriterion c2) {
+        GridTestUtils.assertThrows(null, () -> {
+            IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+                .setCriteria(c1, c2);
+
+            return cache.query(qry).getAll();
+        }, CacheException.class, "Failed to merge criterion");
+    }
+
+    /** */
+    @Test
+    public void testCorrectMergeMultipleBoundaries() {
+        Random rnd = new Random();
+
+        List<Integer> boundaries = IntStream.range(0, 10)
+            .boxed()
+            .map(i -> rnd.nextInt(CNT))
+            .collect(Collectors.toList());
+
+        int min = boundaries.stream().min(Integer::compareTo).get();
+        int max = boundaries.stream().max(Integer::compareTo).get();
+
+        List<IndexQueryCriterion> ltCriteria = boundaries.stream()
+            .map(b -> lt(fldName, b))
+            .collect(Collectors.toList());
+
+        List<IndexQueryCriterion> gtCriteria = boundaries.stream()
+            .map(b -> gt(fldName, b))
+            .collect(Collectors.toList());
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(ltCriteria);
+
+        check(null, cache.query(qry), 0, min);
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(gtCriteria);
+
+        check(null, cache.query(qry), max + 1, CNT);
+    }
+
+    /**
+     * @param left  First cache key, inclusive.
+     * @param right Last cache key, exclusive.
+     */
+    private <T> void check(String errMsg, QueryCursor<Cache.Entry<Integer, Person>> cursor, int left, int right) {
+        List<Cache.Entry<Integer, Person>> all = cursor.getAll();
+
+        assertEquals(errMsg, right - left, all.size());
+
+        Set<Integer> expKeys = IntStream.range(left, right).boxed().collect(Collectors.toSet());
+
+        for (int i = 0; i < all.size(); i++) {
+            Cache.Entry<Integer, Person> entry = all.get(i);
+
+            assertTrue(errMsg, expKeys.remove(entry.getKey()));
+        }
+
+        assertTrue(errMsg, expKeys.isEmpty());
+    }
+
+    /** */
+    private static class Person {
+        /** */
+        @QuerySqlField(orderedGroups = @QuerySqlField.Group(name = ID_IDX, order = 0))
+        final int id;
+
+        /** */
+        @QuerySqlField(orderedGroups = @QuerySqlField.Group(name = DESC_ID_IDX, order = 0, descending = true))
+        final int descId;
+
+        /** */
+        Person(int id) {
+            this.id = id;
+            descId = id;
+        }
+    }
+
+    /** */
+    private List<IndexQueryCriterion> criteria(String fld, int val1, int val2) {
+        return F.asList(
+            eq(fld, val1),
+            lt(fld, val1),
+            lte(fld, val1),
+            gt(fld, val1),
+            gte(fld, val1),
+            between(fld, val1, val2));
+    }
+
+    /** */
+    private void checkTwoCriteria(RangeIndexQueryCriterion c1, RangeIndexQueryCriterion c2) {
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(c1, c2);
+
+        Range expRange = mergeRange(
+            new Range(c1.lower() == null ? 0 : (int)c1.lower(), c1.upper() == null ? CNT : (int)c1.upper(),
+                c1.lowerIncl(), c1.upperIncl()),
+
+            new Range(c2.lower() == null ? 0 : (int)c2.lower(), c2.upper() == null ? CNT : (int)c2.upper(),
+                c2.lowerIncl(), c2.upperIncl()));
+
+        int lower = expRange.lower();
+        int upper = expRange.upper();
+
+        String errMsg = "Fail crit pair: " + c1 + ", " + c2 + ". Lower=" + lower + ", upper=" + upper;
+
+        if (!expRange.valid()) {
+            GridTestUtils.assertThrows(null, () -> cache.query(qry).getAll(),
+                CacheException.class, "Failed to merge criterion",
+                "Not thrown for " + c1 + " " + c2);
+        }
+        else
+            check(errMsg, cache.query(qry), lower, upper);
+    }
+
+    /** */
+    private Range mergeRange(Range range1, Range range2) {
+        int left = range1.left;
+        boolean leftIncl = range1.leftIncl;
+
+        if (range2.left > left) {
+            left = range2.left;
+            leftIncl = range2.leftIncl;
+        } else if (range2.left == left)
+            leftIncl = leftIncl && range2.leftIncl;
+
+        int right = range1.right;
+        boolean rightIncl = range1.rightIncl;
+
+        if (range2.right < right) {
+            right = range2.right;
+            rightIncl = range2.rightIncl;
+        } else if (range2.right == right)
+            rightIncl = rightIncl && range2.rightIncl;
+
+        return new Range(left, right, leftIncl, rightIncl);
+    }
+
+    /** */
+    private static class Range {
+        /** */
+        final int left;
+
+        /** */
+        final int right;
+
+        /** */
+        final boolean leftIncl;
+
+        /** */
+        final boolean rightIncl;
+
+        /** */
+        Range(int left, int right, boolean leftIncl, boolean rightIncl) {
+            this.left = left;
+            this.right = right;
+            this.leftIncl = leftIncl;
+            this.rightIncl = rightIncl;
+        }
+
+        /** */
+        int lower() {
+            return leftIncl ? left : left + 1;
+        }
+
+        /** */
+        int upper() {
+            return rightIncl ? right == CNT ? CNT : right + 1 : right;
+        }
+
+        /** */
+        boolean valid() {
+            return left < right || (left == right && leftIncl && rightIncl);
+        }
+    }
+}