You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/08/06 15:36:45 UTC

[GitHub] [ignite] AMashenkov commented on a change in pull request #9118: IGNITE-14699 Add IndexQuery API.

AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r684268948



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");
+
+        this.valCls = valCls.getName();
+        this.idxName = idxName;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(IndexQueryCriterion criterion, IndexQueryCriterion... criteria) {
+        List<IndexQueryCriterion> cc = new ArrayList<>();
+
+        cc.add(criterion);
+        cc.addAll(Arrays.asList(criteria));
+
+        validateAndSetCriteria(cc);
+
+        return this;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(List<IndexQueryCriterion> criteria) {

Review comment:
       ```suggestion
       /**
        * Sets conjunction (AND) criteria for index query. 
        *
        * @return {@code this} for chaning.
        */
       public IndexQuery<K, V> setCriteria(List<IndexQueryCriterion> criteria) {
   ```
   Actually method provides nothing. it is setter.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");
+
+        this.valCls = valCls.getName();
+        this.idxName = idxName;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.

Review comment:
       Actually method provides nothing. it is setter.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");
+
+        this.valCls = valCls.getName();
+        this.idxName = idxName;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(IndexQueryCriterion criterion, IndexQueryCriterion... criteria) {
+        List<IndexQueryCriterion> cc = new ArrayList<>();
+
+        cc.add(criterion);
+        cc.addAll(Arrays.asList(criteria));
+
+        validateAndSetCriteria(cc);
+
+        return this;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(List<IndexQueryCriterion> criteria) {
+        validateAndSetCriteria(new ArrayList<>(criteria));
+
+        return this;
+    }
+
+    /** Index query criteria. */
+    public List<IndexQueryCriterion> getCriteria() {
+        return criteria;
+    }
+
+    /** Cache value class. */
+    public String getValueClass() {
+        return valCls;
+    }
+
+    /** Index name. */

Review comment:
       missed @return in Javadoc.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQueryCriteriaBuilder.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+/**
+ * Factory of {@link IndexQueryCriterion} for {@link IndexQuery}.
+ */
+public class IndexQueryCriteriaBuilder {

Review comment:
       Method javadocs have no parameters description.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQueryCriteriaBuilder.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+/**
+ * Factory of {@link IndexQueryCriterion} for {@link IndexQuery}.
+ */
+public class IndexQueryCriteriaBuilder {
+    /** Object to mark a boundary if {@code null} is explicitly specified. */
+    private static final Object NULL = new Null();
+
+    /** Equal To. */
+    public static IndexQueryCriterion eq(String field, Object val) {
+        return between(field, val, val);
+    }
+
+    /** Less Then. */
+    public static IndexQueryCriterion lt(String field, Object val) {
+        A.notNullOrEmpty(field, "field");
+
+        return new RangeIndexQueryCriterion(field, null, wrapNull(val), true, false);
+    }
+
+    /** Less Then or Equal. */
+    public static IndexQueryCriterion lte(String field, Object val) {
+        A.notNullOrEmpty(field, "field");
+
+        return new RangeIndexQueryCriterion(field, null, wrapNull(val), true, true);
+    }
+
+    /** Greater Then. */
+    public static IndexQueryCriterion gt(String field, Object val) {
+        A.notNullOrEmpty(field, "field");
+
+        return new RangeIndexQueryCriterion(field, wrapNull(val), null, false, true);
+    }
+
+    /** Greater Then or Equal. */
+    public static IndexQueryCriterion gte(String field, Object val) {
+        A.notNullOrEmpty(field, "field");
+
+        return new RangeIndexQueryCriterion(field, wrapNull(val), null, true, true);
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public static IndexQueryCriterion between(String field, Object lower, Object upper) {
+        A.notNullOrEmpty(field, "field");
+
+        return new RangeIndexQueryCriterion(field, wrapNull(lower), wrapNull(upper), true, true);
+    }
+
+    /** */
+    private static Object wrapNull(Object val) {
+        return val == null ? NULL : val;
+    }
+
+    /** Class to represent NULL value. */
+    public static final class Null {}

Review comment:
       Is this class really needed?
   Seems, absent bound and inclusive 'null' bound will lead to the same result.
   Absent exclusive bound make no sense.
   So, you need to distinct 'null' exclusive case that can perfectly done with the flag.
   

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");
+
+        this.valCls = valCls.getName();
+        this.idxName = idxName;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(IndexQueryCriterion criterion, IndexQueryCriterion... criteria) {

Review comment:
       Why don't just a 
   'public IndexQuery<K, V> setCriteria(IndexQueryCriterion... criteria)'

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);

Review comment:
       Null also can be returned when cache wasn't found...

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);
+
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc.criteria(), qryCtx);
+
+        // Map IndexRow to Cache Key-Value pair.
+        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+            private IndexRow currVal;
+
+            private final CacheObjectContext coctx = cctx.cacheObjectContext();
+
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                if (currVal != null)
+                    return true;
+
+                if (!cursor.next())
+                    return false;
+
+                currVal = cursor.get();
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected IgniteBiTuple<K, V> onNext() {
+                if (currVal == null)
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                IndexRow row = currVal;
+
+                currVal = null;
+
+                K k = (K) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().key(), keepBinary, false);
+                V v = (V) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().value(), keepBinary, false);
+
+                return new IgniteBiTuple<>(k, v);
+            }
+        };
+    }
+
+    /** Get index to run query by specified description. */
+    private Index index(GridCacheContext cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        Class<?> valCls = idxQryDesc.valCls() != null ? loadValClass(cctx, idxQryDesc.valCls()) : null;
+
+        String tableName = cctx.kernalContext().query().tableName(cctx.name(), valCls);
+
+        if (tableName == null)
+            return null;
+
+        // Find index by specified name.
+        if (idxQryDesc.idxName() != null) {
+            String name = idxQryDesc.idxName();
+
+            if (!QueryUtils.PRIMARY_KEY_INDEX.equals(name))
+                name = name.toUpperCase();
+
+            String schema = cctx.kernalContext().query().schemaName(cctx);
+
+            IndexName idxName = new IndexName(cctx.name(), schema, tableName, name);
+
+            Index idx = idxProc.index(idxName);
+
+            if (idx == null)
+                return null;
+
+            return checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.
+        // Check all indexes by cache to find best index match: count of index fields equals to count of index criteria fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            if (checkIndex(idxDef, idxQryDesc.criteria()))
+                return i;
+        }
+
+        return null;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, List<IndexQueryCriterion> criteria) {
+        if (criteria.size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.get(i).field()))

Review comment:
       What if criteria field condition order differs from index?
   Seems, order in criteria make no sense.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");

Review comment:
       Let's introduce another function or use F.isEmpty() with explicit exception throwing.
   Now, user can get "must not be null or empty" message, but the 'null' value is allowed.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");
+
+        this.valCls = valCls.getName();
+        this.idxName = idxName;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(IndexQueryCriterion criterion, IndexQueryCriterion... criteria) {
+        List<IndexQueryCriterion> cc = new ArrayList<>();
+
+        cc.add(criterion);
+        cc.addAll(Arrays.asList(criteria));
+
+        validateAndSetCriteria(cc);
+
+        return this;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(List<IndexQueryCriterion> criteria) {
+        validateAndSetCriteria(new ArrayList<>(criteria));
+
+        return this;
+    }
+
+    /** Index query criteria. */
+    public List<IndexQueryCriterion> getCriteria() {

Review comment:
       Missed @return in Javadoc.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.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.internal.A;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index query runs over internal index structure and returns cache entries for index rows that match specified criteria.
+ */
+@IgniteExperimental
+public final class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index query criteria. */
+    private List<IndexQueryCriterion> criteria;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. */
+    private final @Nullable String idxName;
+
+    /**
+     * Specify index with cache value class.
+     *
+     * @param valCls Cache value class.
+     */
+    public IndexQuery(Class<V> valCls) {
+        this(valCls, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name. If {@code idxName} is {@code null} then Ignite checks
+     * all indexes to find best match by {@link #valCls} and {@link #criteria} fields.
+     *
+     * @param valCls Cache value class.
+     * @param idxName Optional Index name.
+     */
+    public IndexQuery(Class<V> valCls, @Nullable String idxName) {
+        A.notNull(valCls, "valCls");
+
+        if (idxName != null)
+            A.notNullOrEmpty(idxName, "idxName");
+
+        this.valCls = valCls.getName();
+        this.idxName = idxName;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(IndexQueryCriterion criterion, IndexQueryCriterion... criteria) {
+        List<IndexQueryCriterion> cc = new ArrayList<>();
+
+        cc.add(criterion);
+        cc.addAll(Arrays.asList(criteria));
+
+        validateAndSetCriteria(cc);
+
+        return this;
+    }
+
+    /**
+     * Provide multiple index query criterion joint with AND.
+     */
+    public IndexQuery<K, V> setCriteria(List<IndexQueryCriterion> criteria) {
+        validateAndSetCriteria(new ArrayList<>(criteria));
+
+        return this;
+    }
+
+    /** Index query criteria. */
+    public List<IndexQueryCriterion> getCriteria() {
+        return criteria;
+    }
+
+    /** Cache value class. */

Review comment:
       Missed @return in Javadoc.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);
+
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc.criteria(), qryCtx);
+
+        // Map IndexRow to Cache Key-Value pair.
+        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+            private IndexRow currVal;
+
+            private final CacheObjectContext coctx = cctx.cacheObjectContext();
+
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                if (currVal != null)
+                    return true;
+
+                if (!cursor.next())
+                    return false;
+
+                currVal = cursor.get();
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected IgniteBiTuple<K, V> onNext() {
+                if (currVal == null)
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                IndexRow row = currVal;
+
+                currVal = null;
+
+                K k = (K) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().key(), keepBinary, false);
+                V v = (V) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().value(), keepBinary, false);
+
+                return new IgniteBiTuple<>(k, v);
+            }
+        };
+    }
+
+    /** Get index to run query by specified description. */
+    private Index index(GridCacheContext cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        Class<?> valCls = idxQryDesc.valCls() != null ? loadValClass(cctx, idxQryDesc.valCls()) : null;
+
+        String tableName = cctx.kernalContext().query().tableName(cctx.name(), valCls);
+
+        if (tableName == null)
+            return null;
+
+        // Find index by specified name.
+        if (idxQryDesc.idxName() != null) {
+            String name = idxQryDesc.idxName();
+
+            if (!QueryUtils.PRIMARY_KEY_INDEX.equals(name))
+                name = name.toUpperCase();
+
+            String schema = cctx.kernalContext().query().schemaName(cctx);
+
+            IndexName idxName = new IndexName(cctx.name(), schema, tableName, name);
+
+            Index idx = idxProc.index(idxName);
+
+            if (idx == null)
+                return null;
+
+            return checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.
+        // Check all indexes by cache to find best index match: count of index fields equals to count of index criteria fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            if (checkIndex(idxDef, idxQryDesc.criteria()))
+                return i;
+        }
+
+        return null;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, List<IndexQueryCriterion> criteria) {
+        if (criteria.size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.get(i).field()))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    private Class<?> loadValClass(GridCacheContext cctx, String valClsName) throws IgniteCheckedException {
+        try {
+            ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config());
+
+            return clsLdr.loadClass(valClsName);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("No cache serves class: " + valClsName);
+        }
+    }
+
+    /** Runs a query and return single cursor or cursor over multiple index segments. */
+    private GridCursor<IndexRow> query(GridCacheContext cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+
+        if (segmentsCnt == 1)
+            return query(0, idx, criteria, qryCtx);
+
+        final GridCursor<IndexRow>[] segments = new GridCursor[segmentsCnt];
+
+        // Actually it just traverse BPlusTree to find boundaries. It's too fast to parallelize this.
+        for (int i = 0; i < segmentsCnt; i++)
+            segments[i] = query(i, idx, criteria, qryCtx);
+
+        return new SegmentedIndexCursor(segments, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+    }
+
+    /** Coordinate query criteria. */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        assert !criteria.isEmpty() : "Index query criteria list has not to be empty.";
+
+        if (criteria.get(0) instanceof RangeIndexQueryCriterion)
+            return treeIndexRange((InlineIndex) idx, criteria, segment, qryCtx);
+
+        throw new IllegalStateException("Doesn't support index query criteria: " + criteria.getClass().getName());
+    }
+
+    /**
+     * 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(InlineIndex idx, List<IndexQueryCriterion> criteria, int segment,
+        IndexQueryContext qryCtx) throws IgniteCheckedException {
+
+        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
+        CacheObjectContext coctx = idx.segment(0).cacheContext().cacheObjectContext();
+
+        IndexKey[] lowerBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+        IndexKey[] upperBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+
+        boolean lowerAllNulls = true;
+        boolean upperAllNulls = true;
+
+        List<RangeIndexQueryCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.size(); i++) {
+            String f = criteria.get(i).field();
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            // 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();
+
+            treeCriteria.add(c);
+
+            IndexKey l = key(c.lower(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), def, hnd.indexKeyTypeSettings(), coctx);
+
+            if (l != null)
+                lowerAllNulls = false;
+
+            if (u != null)
+                upperAllNulls = false;
+
+            lowerBounds[i] = l;
+            upperBounds[i] = u;
+        }
+
+        IndexRow lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
+        IndexRow upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+
+        // Step 1. Traverse index.
+        GridCursor<IndexRow> findRes = idx.find(lower, upper, segment, qryCtx);
+
+        // Step 2. Scan and filter.
+        return new GridCursor<IndexRow>() {
+            /** */
+            private final IndexRowComparator rowCmp = ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator();
+
+            /** {@inheritDoc} */
+            @Override public boolean next() throws IgniteCheckedException {
+                if (!findRes.next())
+                    return false;
+
+                while (exclude(get(), lower, 1) || exclude(get(), upper, -1)) {
+                    if (!findRes.next())
+                        return false;
+                }
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override public IndexRow get() throws IgniteCheckedException {
+                return findRes.get();
+            }
+
+            /**
+             * Matches index row, boundary and inclusion mask to decide whether this row will be excluded from result.
+             *
+             * @param row Result row to check.
+             * @param boundary Index search boundary.
+             * @param boundarySign {@code 1} for lower boundary and {@code -1} for upper boundary.
+             * @return {@code true} if specified row has to be excluded from result.
+             */
+            private boolean exclude(IndexRow row, IndexRow boundary, int boundarySign) throws IgniteCheckedException {
+                // Unbounded search, include all.
+                if (boundary == null)
+                    return false;
+
+                int criteriaKeysCnt = treeCriteria.size();
+
+                for (int i = 0; i < criteriaKeysCnt; i++) {
+                    RangeIndexQueryCriterion c = treeCriteria.get(i);
+
+                    // Include all values on this field.
+                    if (boundary.key(i) == null)
+                        continue;
+
+                    int cmp = rowCmp.compareKey(row, boundary, i);
+
+                    // Swap direction.
+                    if (hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC)
+                        cmp = -1 * cmp;

Review comment:
       cmp = -cmp

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);
+
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc.criteria(), qryCtx);
+
+        // Map IndexRow to Cache Key-Value pair.
+        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+            private IndexRow currVal;
+
+            private final CacheObjectContext coctx = cctx.cacheObjectContext();
+
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                if (currVal != null)
+                    return true;
+
+                if (!cursor.next())
+                    return false;
+
+                currVal = cursor.get();
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected IgniteBiTuple<K, V> onNext() {
+                if (currVal == null)
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                IndexRow row = currVal;
+
+                currVal = null;
+
+                K k = (K) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().key(), keepBinary, false);
+                V v = (V) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().value(), keepBinary, false);
+
+                return new IgniteBiTuple<>(k, v);
+            }
+        };
+    }
+
+    /** Get index to run query by specified description. */
+    private Index index(GridCacheContext cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        Class<?> valCls = idxQryDesc.valCls() != null ? loadValClass(cctx, idxQryDesc.valCls()) : null;
+
+        String tableName = cctx.kernalContext().query().tableName(cctx.name(), valCls);
+
+        if (tableName == null)
+            return null;
+
+        // Find index by specified name.
+        if (idxQryDesc.idxName() != null) {
+            String name = idxQryDesc.idxName();
+
+            if (!QueryUtils.PRIMARY_KEY_INDEX.equals(name))
+                name = name.toUpperCase();
+
+            String schema = cctx.kernalContext().query().schemaName(cctx);
+
+            IndexName idxName = new IndexName(cctx.name(), schema, tableName, name);
+
+            Index idx = idxProc.index(idxName);
+
+            if (idx == null)
+                return null;
+
+            return checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.
+        // Check all indexes by cache to find best index match: count of index fields equals to count of index criteria fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            if (checkIndex(idxDef, idxQryDesc.criteria()))
+                return i;
+        }
+
+        return null;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, List<IndexQueryCriterion> criteria) {
+        if (criteria.size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.get(i).field()))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    private Class<?> loadValClass(GridCacheContext cctx, String valClsName) throws IgniteCheckedException {
+        try {
+            ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config());
+
+            return clsLdr.loadClass(valClsName);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("No cache serves class: " + valClsName);
+        }
+    }
+
+    /** Runs a query and return single cursor or cursor over multiple index segments. */
+    private GridCursor<IndexRow> query(GridCacheContext cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+
+        if (segmentsCnt == 1)
+            return query(0, idx, criteria, qryCtx);
+
+        final GridCursor<IndexRow>[] segments = new GridCursor[segmentsCnt];
+
+        // Actually it just traverse BPlusTree to find boundaries. It's too fast to parallelize this.
+        for (int i = 0; i < segmentsCnt; i++)
+            segments[i] = query(i, idx, criteria, qryCtx);
+
+        return new SegmentedIndexCursor(segments, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+    }
+
+    /** Coordinate query criteria. */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        assert !criteria.isEmpty() : "Index query criteria list has not to be empty.";
+
+        if (criteria.get(0) instanceof RangeIndexQueryCriterion)
+            return treeIndexRange((InlineIndex) idx, criteria, segment, qryCtx);
+
+        throw new IllegalStateException("Doesn't support index query criteria: " + criteria.getClass().getName());
+    }
+
+    /**
+     * 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(InlineIndex idx, List<IndexQueryCriterion> criteria, int segment,
+        IndexQueryContext qryCtx) throws IgniteCheckedException {
+
+        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
+        CacheObjectContext coctx = idx.segment(0).cacheContext().cacheObjectContext();
+
+        IndexKey[] lowerBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+        IndexKey[] upperBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+
+        boolean lowerAllNulls = true;
+        boolean upperAllNulls = true;
+
+        List<RangeIndexQueryCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.size(); i++) {
+            String f = criteria.get(i).field();
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            // 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();
+
+            treeCriteria.add(c);
+
+            IndexKey l = key(c.lower(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), def, hnd.indexKeyTypeSettings(), coctx);
+
+            if (l != null)
+                lowerAllNulls = false;
+
+            if (u != null)
+                upperAllNulls = false;
+
+            lowerBounds[i] = l;
+            upperBounds[i] = u;
+        }
+
+        IndexRow lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
+        IndexRow upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+
+        // Step 1. Traverse index.
+        GridCursor<IndexRow> findRes = idx.find(lower, upper, segment, qryCtx);
+
+        // Step 2. Scan and filter.
+        return new GridCursor<IndexRow>() {
+            /** */
+            private final IndexRowComparator rowCmp = ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator();
+
+            /** {@inheritDoc} */
+            @Override public boolean next() throws IgniteCheckedException {
+                if (!findRes.next())
+                    return false;
+
+                while (exclude(get(), lower, 1) || exclude(get(), upper, -1)) {
+                    if (!findRes.next())
+                        return false;
+                }
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override public IndexRow get() throws IgniteCheckedException {
+                return findRes.get();
+            }
+
+            /**
+             * Matches index row, boundary and inclusion mask to decide whether this row will be excluded from result.
+             *
+             * @param row Result row to check.
+             * @param boundary Index search boundary.
+             * @param boundarySign {@code 1} for lower boundary and {@code -1} for upper boundary.
+             * @return {@code true} if specified row has to be excluded from result.
+             */
+            private boolean exclude(IndexRow row, IndexRow boundary, int boundarySign) throws IgniteCheckedException {
+                // Unbounded search, include all.
+                if (boundary == null)
+                    return false;
+
+                int criteriaKeysCnt = treeCriteria.size();
+
+                for (int i = 0; i < criteriaKeysCnt; i++) {
+                    RangeIndexQueryCriterion c = treeCriteria.get(i);
+
+                    // Include all values on this field.
+                    if (boundary.key(i) == null)
+                        continue;
+
+                    int cmp = rowCmp.compareKey(row, boundary, i);
+
+                    // Swap direction.
+                    if (hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC)
+                        cmp = -1 * cmp;
+
+                    // Exclude if field equals boundary field and criteria is excluding.
+                    if (cmp == 0) {
+                        if (boundarySign > 0 && !c.lowerIncl())
+                            return true;
+
+                        if (boundarySign < 0 && !c.upperIncl())
+                            return true;
+                    }
+
+                    // Check sign. Exclude if field is out of boundaries.
+                    if (cmp * boundarySign < 0)
+                        return true;
+                }
+
+                return false;
+            }
+        };
+    }
+
+    /** */
+    private IndexKey key(Object val, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null) {
+            if (val instanceof IndexQueryCriteriaBuilder.Null)

Review comment:
       The expression 'x instanceof SomeClass' is false if x is null.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);
+
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc.criteria(), qryCtx);
+
+        // Map IndexRow to Cache Key-Value pair.
+        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+            private IndexRow currVal;
+
+            private final CacheObjectContext coctx = cctx.cacheObjectContext();
+
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                if (currVal != null)
+                    return true;
+
+                if (!cursor.next())
+                    return false;
+
+                currVal = cursor.get();
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected IgniteBiTuple<K, V> onNext() {
+                if (currVal == null)
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                IndexRow row = currVal;
+
+                currVal = null;
+
+                K k = (K) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().key(), keepBinary, false);
+                V v = (V) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().value(), keepBinary, false);
+
+                return new IgniteBiTuple<>(k, v);
+            }
+        };
+    }
+
+    /** Get index to run query by specified description. */
+    private Index index(GridCacheContext cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        Class<?> valCls = idxQryDesc.valCls() != null ? loadValClass(cctx, idxQryDesc.valCls()) : null;
+
+        String tableName = cctx.kernalContext().query().tableName(cctx.name(), valCls);
+
+        if (tableName == null)
+            return null;
+
+        // Find index by specified name.
+        if (idxQryDesc.idxName() != null) {
+            String name = idxQryDesc.idxName();
+
+            if (!QueryUtils.PRIMARY_KEY_INDEX.equals(name))
+                name = name.toUpperCase();
+
+            String schema = cctx.kernalContext().query().schemaName(cctx);
+
+            IndexName idxName = new IndexName(cctx.name(), schema, tableName, name);
+
+            Index idx = idxProc.index(idxName);
+
+            if (idx == null)
+                return null;
+
+            return checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.
+        // Check all indexes by cache to find best index match: count of index fields equals to count of index criteria fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            if (checkIndex(idxDef, idxQryDesc.criteria()))
+                return i;
+        }
+
+        return null;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, List<IndexQueryCriterion> criteria) {
+        if (criteria.size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.get(i).field()))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    private Class<?> loadValClass(GridCacheContext cctx, String valClsName) throws IgniteCheckedException {
+        try {
+            ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config());
+
+            return clsLdr.loadClass(valClsName);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("No cache serves class: " + valClsName);
+        }
+    }
+
+    /** Runs a query and return single cursor or cursor over multiple index segments. */
+    private GridCursor<IndexRow> query(GridCacheContext cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+
+        if (segmentsCnt == 1)
+            return query(0, idx, criteria, qryCtx);
+
+        final GridCursor<IndexRow>[] segments = new GridCursor[segmentsCnt];
+
+        // Actually it just traverse BPlusTree to find boundaries. It's too fast to parallelize this.
+        for (int i = 0; i < segmentsCnt; i++)
+            segments[i] = query(i, idx, criteria, qryCtx);
+
+        return new SegmentedIndexCursor(segments, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+    }
+
+    /** Coordinate query criteria. */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        assert !criteria.isEmpty() : "Index query criteria list has not to be empty.";
+
+        if (criteria.get(0) instanceof RangeIndexQueryCriterion)
+            return treeIndexRange((InlineIndex) idx, criteria, segment, qryCtx);
+
+        throw new IllegalStateException("Doesn't support index query criteria: " + criteria.getClass().getName());
+    }
+
+    /**
+     * 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(InlineIndex idx, List<IndexQueryCriterion> criteria, int segment,
+        IndexQueryContext qryCtx) throws IgniteCheckedException {
+
+        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
+        CacheObjectContext coctx = idx.segment(0).cacheContext().cacheObjectContext();
+
+        IndexKey[] lowerBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+        IndexKey[] upperBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+
+        boolean lowerAllNulls = true;
+        boolean upperAllNulls = true;
+
+        List<RangeIndexQueryCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.size(); i++) {
+            String f = criteria.get(i).field();
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            // 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();
+
+            treeCriteria.add(c);
+
+            IndexKey l = key(c.lower(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), def, hnd.indexKeyTypeSettings(), coctx);
+
+            if (l != null)
+                lowerAllNulls = false;
+
+            if (u != null)
+                upperAllNulls = false;
+
+            lowerBounds[i] = l;
+            upperBounds[i] = u;
+        }
+
+        IndexRow lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
+        IndexRow upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+
+        // Step 1. Traverse index.
+        GridCursor<IndexRow> findRes = idx.find(lower, upper, segment, qryCtx);
+
+        // Step 2. Scan and filter.
+        return new GridCursor<IndexRow>() {
+            /** */
+            private final IndexRowComparator rowCmp = ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator();
+
+            /** {@inheritDoc} */
+            @Override public boolean next() throws IgniteCheckedException {
+                if (!findRes.next())
+                    return false;
+
+                while (exclude(get(), lower, 1) || exclude(get(), upper, -1)) {
+                    if (!findRes.next())
+                        return false;
+                }
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override public IndexRow get() throws IgniteCheckedException {
+                return findRes.get();
+            }
+
+            /**
+             * Matches index row, boundary and inclusion mask to decide whether this row will be excluded from result.
+             *
+             * @param row Result row to check.
+             * @param boundary Index search boundary.
+             * @param boundarySign {@code 1} for lower boundary and {@code -1} for upper boundary.
+             * @return {@code true} if specified row has to be excluded from result.
+             */
+            private boolean exclude(IndexRow row, IndexRow boundary, int boundarySign) throws IgniteCheckedException {
+                // Unbounded search, include all.
+                if (boundary == null)
+                    return false;
+
+                int criteriaKeysCnt = treeCriteria.size();
+
+                for (int i = 0; i < criteriaKeysCnt; i++) {
+                    RangeIndexQueryCriterion c = treeCriteria.get(i);
+
+                    // Include all values on this field.
+                    if (boundary.key(i) == null)
+                        continue;
+
+                    int cmp = rowCmp.compareKey(row, boundary, i);
+
+                    // Swap direction.
+                    if (hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC)
+                        cmp = -1 * cmp;
+
+                    // Exclude if field equals boundary field and criteria is excluding.
+                    if (cmp == 0) {
+                        if (boundarySign > 0 && !c.lowerIncl())
+                            return true;
+
+                        if (boundarySign < 0 && !c.upperIncl())
+                            return true;
+                    }
+
+                    // Check sign. Exclude if field is out of boundaries.
+                    if (cmp * boundarySign < 0)
+                        return true;
+                }
+
+                return false;
+            }
+        };
+    }
+
+    /** */
+    private IndexKey key(Object val, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null) {
+            if (val instanceof IndexQueryCriteriaBuilder.Null)
+                val = null;
+
+            key = IndexKeyFactory.wrap(
+                val, def.idxType(), coctx, settings);
+        }
+
+        return key;
+    }
+
+    /** Single cursor over multiple segments. Next value is choose with the index row comparator. */
+    private class SegmentedIndexCursor implements GridCursor<IndexRow> {
+        /** Cursors over segments. */
+        private final GridCursor<IndexRow>[] cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Offset of current segmented cursor to return value. */
+        private int cursorOff;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            this.cursors = cursors;
+
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (o1 == o2)
+                            return 0;
+
+                        if (o1 == null)
+                            return -1;
+
+                        if (o2 == null)
+                            return 1;

Review comment:
       Does 'nulls' order independent from index sort order?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);
+
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc.criteria(), qryCtx);
+
+        // Map IndexRow to Cache Key-Value pair.
+        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+            private IndexRow currVal;
+
+            private final CacheObjectContext coctx = cctx.cacheObjectContext();
+
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                if (currVal != null)
+                    return true;
+
+                if (!cursor.next())
+                    return false;
+
+                currVal = cursor.get();
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected IgniteBiTuple<K, V> onNext() {
+                if (currVal == null)
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                IndexRow row = currVal;
+
+                currVal = null;
+
+                K k = (K) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().key(), keepBinary, false);
+                V v = (V) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().value(), keepBinary, false);
+
+                return new IgniteBiTuple<>(k, v);
+            }
+        };
+    }
+
+    /** Get index to run query by specified description. */
+    private Index index(GridCacheContext cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        Class<?> valCls = idxQryDesc.valCls() != null ? loadValClass(cctx, idxQryDesc.valCls()) : null;
+
+        String tableName = cctx.kernalContext().query().tableName(cctx.name(), valCls);
+
+        if (tableName == null)
+            return null;
+
+        // Find index by specified name.
+        if (idxQryDesc.idxName() != null) {
+            String name = idxQryDesc.idxName();
+
+            if (!QueryUtils.PRIMARY_KEY_INDEX.equals(name))
+                name = name.toUpperCase();
+
+            String schema = cctx.kernalContext().query().schemaName(cctx);
+
+            IndexName idxName = new IndexName(cctx.name(), schema, tableName, name);
+
+            Index idx = idxProc.index(idxName);
+
+            if (idx == null)
+                return null;
+
+            return checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.
+        // Check all indexes by cache to find best index match: count of index fields equals to count of index criteria fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            if (checkIndex(idxDef, idxQryDesc.criteria()))
+                return i;
+        }
+
+        return null;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, List<IndexQueryCriterion> criteria) {
+        if (criteria.size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.get(i).field()))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    private Class<?> loadValClass(GridCacheContext cctx, String valClsName) throws IgniteCheckedException {
+        try {
+            ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config());
+
+            return clsLdr.loadClass(valClsName);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("No cache serves class: " + valClsName);
+        }
+    }
+
+    /** Runs a query and return single cursor or cursor over multiple index segments. */
+    private GridCursor<IndexRow> query(GridCacheContext cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+
+        if (segmentsCnt == 1)
+            return query(0, idx, criteria, qryCtx);
+
+        final GridCursor<IndexRow>[] segments = new GridCursor[segmentsCnt];
+
+        // Actually it just traverse BPlusTree to find boundaries. It's too fast to parallelize this.
+        for (int i = 0; i < segmentsCnt; i++)
+            segments[i] = query(i, idx, criteria, qryCtx);
+
+        return new SegmentedIndexCursor(segments, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+    }
+
+    /** Coordinate query criteria. */

Review comment:
       Confusing javadoc.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,449 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.cache.query.IndexQueryCriterion;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+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.IndexRow;
+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.inline.IndexQueryContext;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+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;
+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.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+import static org.apache.ignite.internal.cache.query.index.SortOrder.DESC;
+
+/**
+ * Processor of {@link IndexQuery}.
+ */
+public class IndexQueryProcessor {
+    /** */
+    private final IndexProcessor idxProc;
+
+    /** */
+    public IndexQueryProcessor(IndexProcessor idxProc) {
+        this.idxProc = idxProc;
+    }
+
+    /** Run query on local node. */
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocal(
+        GridCacheContext<K, V> cctx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx, boolean keepBinary)
+        throws IgniteCheckedException {
+
+        Index idx = index(cctx, idxQryDesc);
+
+        if (idx == null)
+            throw new IgniteCheckedException(
+                "No index matches index query. Cache=" + cctx.name() + "; Qry=" + idxQryDesc);
+
+        GridCursor<IndexRow> cursor = query(cctx, idx, idxQryDesc.criteria(), qryCtx);
+
+        // Map IndexRow to Cache Key-Value pair.
+        return new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
+            private IndexRow currVal;
+
+            private final CacheObjectContext coctx = cctx.cacheObjectContext();
+
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                if (currVal != null)
+                    return true;
+
+                if (!cursor.next())
+                    return false;
+
+                currVal = cursor.get();
+
+                return true;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected IgniteBiTuple<K, V> onNext() {
+                if (currVal == null)
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                IndexRow row = currVal;
+
+                currVal = null;
+
+                K k = (K) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().key(), keepBinary, false);
+                V v = (V) CacheObjectUtils.unwrapBinaryIfNeeded(coctx, row.cacheDataRow().value(), keepBinary, false);
+
+                return new IgniteBiTuple<>(k, v);
+            }
+        };
+    }
+
+    /** Get index to run query by specified description. */
+    private Index index(GridCacheContext cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        Class<?> valCls = idxQryDesc.valCls() != null ? loadValClass(cctx, idxQryDesc.valCls()) : null;
+
+        String tableName = cctx.kernalContext().query().tableName(cctx.name(), valCls);
+
+        if (tableName == null)
+            return null;
+
+        // Find index by specified name.
+        if (idxQryDesc.idxName() != null) {
+            String name = idxQryDesc.idxName();
+
+            if (!QueryUtils.PRIMARY_KEY_INDEX.equals(name))
+                name = name.toUpperCase();
+
+            String schema = cctx.kernalContext().query().schemaName(cctx);
+
+            IndexName idxName = new IndexName(cctx.name(), schema, tableName, name);
+
+            Index idx = idxProc.index(idxName);
+
+            if (idx == null)
+                return null;
+
+            return checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.
+        // Check all indexes by cache to find best index match: count of index fields equals to count of index criteria fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            if (checkIndex(idxDef, idxQryDesc.criteria()))
+                return i;
+        }
+
+        return null;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, List<IndexQueryCriterion> criteria) {
+        if (criteria.size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.get(i).field()))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    private Class<?> loadValClass(GridCacheContext cctx, String valClsName) throws IgniteCheckedException {
+        try {
+            ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config());
+
+            return clsLdr.loadClass(valClsName);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("No cache serves class: " + valClsName);
+        }
+    }
+
+    /** Runs a query and return single cursor or cursor over multiple index segments. */
+    private GridCursor<IndexRow> query(GridCacheContext cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
+
+        if (segmentsCnt == 1)
+            return query(0, idx, criteria, qryCtx);
+
+        final GridCursor<IndexRow>[] segments = new GridCursor[segmentsCnt];
+
+        // Actually it just traverse BPlusTree to find boundaries. It's too fast to parallelize this.
+        for (int i = 0; i < segmentsCnt; i++)
+            segments[i] = query(i, idx, criteria, qryCtx);
+
+        return new SegmentedIndexCursor(segments, ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator());
+    }
+
+    /** Coordinate query criteria. */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        assert !criteria.isEmpty() : "Index query criteria list has not to be empty.";
+
+        if (criteria.get(0) instanceof RangeIndexQueryCriterion)
+            return treeIndexRange((InlineIndex) idx, criteria, segment, qryCtx);
+
+        throw new IllegalStateException("Doesn't support index query criteria: " + criteria.getClass().getName());
+    }
+
+    /**
+     * 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(InlineIndex idx, List<IndexQueryCriterion> criteria, int segment,
+        IndexQueryContext qryCtx) throws IgniteCheckedException {
+
+        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
+        CacheObjectContext coctx = idx.segment(0).cacheContext().cacheObjectContext();
+
+        IndexKey[] lowerBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+        IndexKey[] upperBounds = new IndexKey[hnd.indexKeyDefinitions().size()];
+
+        boolean lowerAllNulls = true;
+        boolean upperAllNulls = true;
+
+        List<RangeIndexQueryCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.size(); i++) {
+            String f = criteria.get(i).field();
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            // 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();
+
+            treeCriteria.add(c);
+
+            IndexKey l = key(c.lower(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), def, hnd.indexKeyTypeSettings(), coctx);
+
+            if (l != null)
+                lowerAllNulls = false;
+
+            if (u != null)
+                upperAllNulls = false;
+
+            lowerBounds[i] = l;
+            upperBounds[i] = u;
+        }
+
+        IndexRow lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
+        IndexRow upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+
+        // Step 1. Traverse index.
+        GridCursor<IndexRow> findRes = idx.find(lower, upper, segment, qryCtx);
+
+        // Step 2. Scan and filter.
+        return new GridCursor<IndexRow>() {
+            /** */
+            private final IndexRowComparator rowCmp = ((SortedIndexDefinition) idxProc.indexDefinition(idx.id())).rowComparator();
+
+            /** {@inheritDoc} */
+            @Override public boolean next() throws IgniteCheckedException {
+                if (!findRes.next())
+                    return false;
+
+                while (exclude(get(), lower, 1) || exclude(get(), upper, -1)) {
+                    if (!findRes.next())
+                        return false;
+                }

Review comment:
       This looks ambiguous.
   
   All row in index are sorted and OR conditions are not allowed.
   So, you just need to skip rows that are below a lower bound and then just detect when an upper bound is reached, is it right?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org