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/05/21 17:57:33 UTC

[GitHub] [ignite] timoninmaxim opened a new pull request #9118: IGNITE-14699 Add IndexQuery API.

timoninmaxim opened a new pull request #9118:
URL: https://github.com/apache/ignite/pull/9118


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696461174



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       Alternatives are:
   1. Additional storage for initial cursors.
   2. Store prefetched next value with additional `head` variable.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696461174



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       Alternatives are:
   1. Additional storage for initial cursors.
   2. Store next value with `head` variable.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660532798



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.IndexQueryCriteria;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriteria;
+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);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.criteria())) {
+                if (idx == null) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else if (fldsCnt < idxFieldsCnt) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else continue;
+
+                // Best match. Index query criteria matches full index.
+                if (idxQryDesc.criteria().fields().size() == idxDef.indexKeyDefinitions().size())
+                    break;
+            }
+        }
+
+        return idx;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria) {
+        if (criteria.fields().size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.fields().get(i)))
+                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, IndexQueryCriteria 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, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        if (criteria instanceof RangeIndexQueryCriteria)
+            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) 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, RangeIndexQueryCriteria 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<RangeIndexQueryCriteria.RangeCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            String f = criteria.fields().get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriteria.RangeCriterion c = criteria.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 (match(get(), lower, 1) || match(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 match(IndexRow row, IndexRow boundary, int boundarySign) throws IgniteCheckedException {

Review comment:
       Rename to `exclude`




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685959856



##########
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:
       > Why we need additional checks on A
   
   Actually don't need. I've already implement this improvement in separate branch. But I think, it's pretty simple fix, so I will do it within this PR.
   
   > Also, the rows that returned from the Cursor are materialized
   
   There is already a ticket for that. [IGNITE-14945](https://issues.apache.org/jira/browse/IGNITE-14945). Also I've already implement it, so just need to finish current PR to submit a new patch. 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r643004142



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {

Review comment:
       @AMashenkov hi! I think it is a good idea. I'll do it, but at first we should discuss comment @tledkov-gridgain about ugly the between() method. Solution will affect how the builder will look like.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660594599



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TABLE = "TEST_CACHE_TABLE";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    private Ignite crd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration<>()

Review comment:
       How does it help? All tests (except 2) have different `getConfiguration()`, the only common thing is a cache name. But do we really need a separate class for that? Also flat structure make tests more convenient for debug. 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696430483



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       Did we already swapped boundaries in treeIndexRange() ?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696433034



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);

Review comment:
       It will raise exception during the check
   
   > Caused by: class org.apache.ignite.IgniteCheckedException: Failed to parse IndexQuery. No index matches query. Cache=TEST_CACHE; Query=IndexQuery[idxName=null, valCls=org.apache.ignite.cache.query.IndexQueryFailoverTest$Person, fields=[id, nonExistedField]]
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685788237



##########
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:
       This code is guarded with busyLock, so there is no chance to get NPE. Don't understand what is wrong there.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660516364



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
##########
@@ -377,6 +377,28 @@ public IndexesRebuildTask idxRebuild() {
         }
     }
 
+    /**
+     * Returns index for specified name.
+     *
+     * @param idxName Index name.
+     * @return Index for specified index name.
+     */
+    public Index index(IndexName idxName) {
+        ddlLock.readLock().lock();
+
+        try {
+            Map<String, Index> idxs = cacheToIdx.get(idxName.cacheName());
+
+            if (idxs == null)
+                return null;
+
+            return idxs.get(idxName.fullName());
+
+        } finally {

Review comment:
       Also changed everywhere within this file.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696432436



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);

Review comment:
       There is already a check on that before executing query, see [IndexQueryProcessor#checkIndex](https://github.com/apache/ignite/blob/5ad86ab09f15f2fac28aec5466f332fe7e11a052/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java#L195)




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685860079



##########
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:
       Got it. 
   Assume, there is a composite index on (A,B).
   So, we need to scan contiguous range on A (top level) and filter out B (lower levels).
   
   As I understand, idx.find(...) already returns values that satisfy condition on A, and (as you wrote) don't filter out conditions on B. 
   Why we need additional checks on A? Is this because we may need to filter out rows in case of exclusive bounds?
   If so, Index interface must specify whether bounds are inclusive or exclusive.
   
   Also, the rows that returned from the Cursor are materialized, even if they will be filtered out.
   It looks ok for now, but let's create a ticket for improvement.
   At first glance, IndexKey might have inclusive/exclusive flag, to avoid Row materialization if range has exclusive bounds. And we can think about passing a filter.
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685809550



##########
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:
       For example we have a query `between(A, 1, 2).between(B, 3, 4)`. For this case row (A=1, B=2) is valid for our tree structure. We compare rows starting with most-significant columns. If most-significant column (A) pass comparison then less-significant columns (B) aren't checked. So B=2 is illegal by user query, but legal for our tree structure. So we need additional filter step there.
   




-- 
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



[GitHub] [ignite] AMashenkov merged pull request #9118: IGNITE-14699 Add IndexQuery API.

Posted by GitBox <gi...@apache.org>.
AMashenkov merged pull request #9118:
URL: https://github.com/apache/ignite/pull/9118


   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r698550980



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       Simplified this code. Now we prepare fill the priority queue in constructor, and prepare `head` on `next()` step, and it doesn't violate the contract.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685298084



##########
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:
       Actually it's validated with `validateAndSetCriteria()` within IndexQuery. But I agree that it's possible to set empty List, so there is no need to restrict `setCriteria` with varargs.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660543534



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());

Review comment:
       Don't understand why count of backups should affect result size? Actually we return data from primary nodes only.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696414843



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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];

Review comment:
       ```suggestion
           final GridCursor<IndexRow>[] segmentCursor = new GridCursor[segmentsCnt];
   ```




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660684442



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java
##########
@@ -0,0 +1,598 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String INDEX = "TEST_IDX";
+
+    /** */
+    private static final String DESC_INDEX = "TEST_DESC_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    private Ignite ignite;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Collection<Object[]> testParams() {
+        return Arrays.asList(
+            new Object[] {1},
+            new Object[] {2});
+    }
+
+            /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        ignite = startGrids(nodesCnt);
+
+        cache = ignite.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>()
+            .setName("TEST_CACHE")
+            .setIndexedTypes(Long.class, Person.class)
+            .setQueryParallelism(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testQueryKeyPKIndex() {
+        insertData();
+
+        int pivot = new Random().nextInt(CNT);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, "_key_PK")
+            .setCriteria(lt("_KEY", (long) pivot));
+
+        checkPerson(cache.query(qry), 0, pivot);
+    }
+
+    /** */
+    @Test
+    public void testEmptyCacheQuery() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE), lt("secId", Integer.MAX_VALUE));
+
+        QueryCursor<Cache.Entry<Long, Person>> cursor = cache.query(qry);
+
+        assertTrue(cursor.getAll().isEmpty());
+
+        // Check same query with specify index name.
+        qry = new IndexQuery<Long, Person>(Person.class, INDEX)
+            .setCriteria(lt("id", Integer.MAX_VALUE), lt("secId", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testCheckBoundaries() {
+        cache.put(1L, new Person(0, 1));
+        cache.put(2L, new Person(1, 0));
+        cache.put(3L, new Person(1, 1));
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(between("id", 0, 1), eq("secId", 1));
+
+        List<Cache.Entry<Long, Person>> result = cache.query(qry).getAll();
+
+        assertEquals(2, result.size());
+
+        result.sort(Comparator.comparingLong(Cache.Entry::getKey));
+
+        assertEquals(1L, (long) result.get(0).getKey());
+        assertEquals(3L, (long) result.get(1).getKey());
+
+        assertEquals(new Person(0, 1), result.get(0).getValue());
+        assertEquals(new Person(1, 1), result.get(1).getValue());
+    }
+
+    /** */
+    @Test
+    public void testLtQueryMultipleField() {
+        insertData();
+
+        int pivot = new Random().nextInt(CNT);
+
+        // Should return empty result for ID that less any inserted.
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", -1), lt("secId", pivot));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        // Should return all data for ID and SECID that greater any inserted.

Review comment:
       Greater is related to specified value in query. We set ID = 1 that greater that any inserted (id = 0 for all rows).




-- 
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



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

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r662927526



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -222,11 +204,13 @@ private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria)
     }
 
     /** Coordinate query criteria. */
-    private GridCursor<IndexRow> query(int segment, Index idx, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
         throws IgniteCheckedException {
 
-        if (criteria instanceof RangeIndexQueryCriteria)
-            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) criteria, segment, qryCtx);
+        assert !criteria.isEmpty() : "Index qury criteria list has not to be empty.";

Review comment:
       `Index qury` -> `Index query`




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642557531



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       The logic of choose index by IndexQuery may confuse user in case multiple-filed index. I propose to think about QueryIndex as low-level API for index access. I guess public Index interface and find/scan methods will be more appropriate. 
   
   Let's disallow the creation of a IndexQuery without explicitly specifying the index.
   
   I'll explain my point of view.
   
   The choice of the index is the work of the SKL optimizer. The optimizer can use statistics and other metadata.
   
   Duplicating optimizer logic is a bad idea.
   It may be surprising for the user to get two different indexes in the SQL optimizer and IndexQuery for the same set of conditions.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696432170



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       It would be much cleaner to have a method with signature exclude(row, lower, upper).
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642650320



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       I'm agree that it's better to directly use index name. But currently name of index is an optional parameter. No QueryIndex, no QuerySqlField requires index name. So I believe we can query index by the same rules as we create it - just value class and list of fields. it should be enough.
   
   > Duplicating optimizer logic is a bad idea. It may be surprising for the user to get two different indexes in the SQL optimizer and IndexQuery for the same set of conditions
   
   IndexQuery doesn't promise optimize query. Even more it declares as query that doesn't waste time on planning, optimization, etc. It just get index that best match for condition and run a query on it. So for cases when optimization is required user has to specify index name.
   
   So I think we should use both ways of index definition.
   




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696416566



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       > No QueryIndex, no QuerySqlField requires index name
   
   They include SQL engine, optimizer and cost model under cover. 
   Choose the index by conditions - the work for optimizer.
   
   It looks like the order of the criteria should be the same as the order of the index fields.
   You really means that it is simple to understand for user than specify the index name?
   My point: index name must be specified by the user. IndexAPI must not find appropriate index.
   Lets discuss it on the devlist to get more opinions.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696430627



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);

Review comment:
       > Guess, an out-of-bound exception will be thrown here.
   
   Looks at the check on the line upper: 
    ```
     if (criteria.size() > idxDef.indexKeyDefinitions().size())
               return false;
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);

Review comment:
       @AMashenkov 
   > Guess, an out-of-bound exception will be thrown here.
   
   Looks at the check on the line upper: 
    ```
     if (criteria.size() > idxDef.indexKeyDefinitions().size())
               return false;
   ```




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696414099



##########
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:
       But public API says nothing about this limitations.
   I'm ok to have this limitations in this PR if it will be documented, and I think we should avoid this in future.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685830248



##########
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:
       Here you can get null if table wasn't found, but error message says "No index matches..."
   Index idx = index(cctx, idxQryDesc);
   
   Maybe, an exception with more specific message can be thrown from index() method. WDYT?
   
   I'd suggest add 2 methods: 1-st returning specific index by name, and 2-nd for searching suitable index for the criteria (by field names)




-- 
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



[GitHub] [ignite] alex-plekhanov commented on pull request #9118: IGNITE-14699 Add IndexQuery API.

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#issuecomment-868282915


   Let's also add some test where the index field alias not equal to java field name.


-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696493090



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       > It would be much cleaner to have a method with
   
   The `exclude()` method checks a row with the specified boundary. It's impossible to check both boundaries simultaneously, but it's a sequential job. If we provide method `exclude(row, lower, upper)` it will be just a wrapper on the same condition: `exclude(get(), lower, 1) || exclude(get(), upper, -1)`.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685975984



##########
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:
       Replaced Null with flags. Looks OK.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642650320



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       I'm agree that it's better to directly use index name. But currently name of index is an optional parameter. No QueryIndex, no QuerySqlField requires index name. So I believe we can query index by the same rules as we create it - just value class and list of fields. it should be enough.
   
   > Duplicating optimizer logic is a bad idea. It may be surprising for the user to get two different indexes in the SQL optimizer and IndexQuery for the same set of conditions
   
   IndexQuery doesn't promise optimize query. Even more it declares as IndexQuery doesn't waste time on planning, optimization, etc. It just get index that best match for condition and run a query on it. So for cases when optimization is required user has to specify index name.
   
   So I think we should use both ways of index definition.
   




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696435957



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       This looks like a dirty hack.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685000979



##########
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:
       You have similar method with a List argument and it doesn't force user to specify non-empty list.
   
   Arguments must be validated instead of having confusing API.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r708203541



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       It's possible. There is no magic constant anymore.
   ```
    private boolean rowIsOutOfRange(IndexRow row, IndexRow low, IndexRow high) throws IgniteCheckedException {
           if (low == null && high == null)
               return true; // Unbounded search, include all.
   
           int criteriaKeysCnt = treeCriteria.size();
   
           for (int i = 0; i < criteriaKeysCnt; i++) {
               RangeIndexQueryCriterion c = treeCriteria.get(i);
   
               boolean descOrder = hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC;
   
               if (low != null && low.key(i) != null) {
                   int cmp = rowCmp.compareKey(row, low, i);
   
                   if (cmp == 0) {
                       if (!c.lowerIncl())
                           return true;     // Exclude if field equals boundary field and criteria is excluding.
                   }
                   else if ((cmp < 0) ^ descOrder)
                       return true; // Out of bound. Either below 'low' margin or column with desc order.
               }
   
               if (high != null && low.high(i) != null) {
                   int cmp = rowCmp.compareKey(row, high, i);
              
                   if (cmp == 0) {
                       if (!c.upperIncl())
                           return true;    // Exclude if field equals boundary field and criteria is excluding.
                   }
                   else if ((cmp > 0) ^ descOrder)
                       return true; // Out of bound. Either above 'high' margin or column with desc order.
               }
           }
   
           return false;
       }
   
   ```




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696483713



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       > Did we already swapped boundaries in treeIndexRange() ?
   
   In treeRange we use swap for user's defined criteria - it's required to correctly work with DESC indexes with mirrored criteria (otherwise, for example, gt() for DESC index will work unexpected for user). But it doesn't affect results of comparing. We need correctly work with DESC indexes while compare - consider additional -1 multiplier. InlineIndexTree does this for traversing part, but we also need do it for filter part.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685000979



##########
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:
       You have similar method with a List argument and it doesn't force user to specify non-empty list.
   
   Arguments must be validated instead of offering user a confusing API.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r658293317



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.IndexConditionBuilder;
+import org.apache.ignite.cache.query.IndexQuery;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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.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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? 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 condition fields.
+        Collection<Index> idxs = idxProc.indexes(cctx);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.idxCond())) {

Review comment:
       1. Indexes (a, b, c) and (a, b) have the same selectivity for (a, b) query. And without analyzing inline size it's impossible to say which one is preferred.
   2. It's a strange case, because having both (a, b, c) and (a, b) indexes brings only performance drop, and (a, b) index should be dropped.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685387544



##########
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:
       IndexQuery is delegated to `InlineIndexImpl.find(lower, upper)`. If the find method invoked with (null, null) params it returns whole range, even if there is no any nulls (e.g. [1, 2] is valid range for query (null, null)). But if user wants find only NULL, for example `eq(NULL)` or `lte(NULL)`. In those cases we should to mark user defined NULL. 
   
   We could mark it with some flags instead class, like `strictLowerNull`. But in case of multi-field query it looks bad as we need support array of flags. So I think usage specific class is better solution.




-- 
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



[GitHub] [ignite] timoninmaxim commented on pull request #9118: IGNITE-14699 Add IndexQuery API.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#issuecomment-895993554


   @AMashenkov I've pushed fixes, could you please have a look?


-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685976256



##########
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:
       Make exceptions more detailed.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642557531



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       The logic of choose index by IndexQuery may confuse user in case multiple-filed index. I propose to think about QueryIndex as low-level API for index access. I guess public Index interface and find/scan methods will be more appropriate. 
   
   Let's disallow the creation of a IndexQuery without explicitly specifying the index.
   
   I'll explain my point of view.
   
   The choice of the index is the work of the SQL optimizer. The optimizer can use statistics and other metadata.
   
   Duplicating optimizer logic is a bad idea.
   It may be surprising for the user to get two different indexes in the SQL optimizer and IndexQuery for the same set of conditions.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r684673395



##########
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:
       This is done to force user to specify at least one criteria. `setCriteria()` without params is useless.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r657763218



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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 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 class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {

Review comment:
       Lets make class `final`

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());

Review comment:
       Lets also check that sum of `result.size()` from all nodes equals to `CNT / 2 * (backups + 1)`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
##########
@@ -377,6 +377,28 @@ public IndexesRebuildTask idxRebuild() {
         }
     }
 
+    /**
+     * Returns index for specified name.
+     *
+     * @param idxName Index name.
+     * @return Index for specified index name.
+     */
+    public Index index(IndexName idxName) {
+        ddlLock.readLock().lock();
+
+        try {
+            Map<String, Index> idxs = cacheToIdx.get(idxName.cacheName());
+
+            if (idxs == null)
+                return null;
+
+            return idxs.get(idxName.fullName());
+
+        } finally {

Review comment:
       NL after `}`

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQueryCriteria.java
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Basic interface for all {@link IndexQuery} criteria.
+ */
+public interface IndexQueryCriteria extends Serializable {
+    /**
+     * @return List of fields that this query criteria applies to.
+     */
+    public abstract List<String> fields();

Review comment:
       `abstract` keyword seems redundant.
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
##########
@@ -483,11 +523,18 @@ public ClusterGroup projection() {
         return part;
     }
 
+    /**
+     * @return Index query description.
+     */
+    @Nullable public IndexQueryDesc idxQryDesc() { return idxQryDesc; }
+
     /**
      * @throws IgniteCheckedException If query is invalid.
      */
     public void validate() throws IgniteCheckedException {
-        if ((type != SCAN && type != SET && type != SPI) && !QueryUtils.isEnabled(cctx.config()))
+        if ((type != SCAN && type != SET && type != SPI && type != INDEX)
+            && !QueryUtils.isEnabled(cctx.config()))
+

Review comment:
       redundant NL

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryQueryEntityTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TBL_NAME = "TEST_CACHE_TBL_NAME";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache<Long, Person> cache;
+
+    /** */
+    private IgniteCache<Long, Person> cacheTblName;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+        cacheTblName = crd.cache(CACHE_TBL_NAME);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        QueryIndex idIdx = new QueryIndex("id", true, ID_IDX);
+        QueryIndex descIdIdx = new QueryIndex("descId", false, DESC_ID_IDX);
+
+        QueryEntity e = new QueryEntity(Long.class.getName(), Person.class.getName())
+            .setFields(new LinkedHashMap<>(
+                F.asMap("id", Integer.class.getName(), "descId", Integer.class.getName()))
+            )
+            .setIndexes(Arrays.asList(idIdx, descIdIdx));
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setQueryEntities(Collections.singletonList(e));
+
+        QueryEntity entTableName = new QueryEntity(e);
+        entTableName.setTableName(TABLE);
+
+        CacheConfiguration<?, ?> ccfg2 = new CacheConfiguration<>()
+            .setName(CACHE_TBL_NAME)
+            .setQueryEntities(Collections.singletonList(entTableName));
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("descId", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)

Review comment:
       Why do we need to check this and the previous query twice?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.IndexQueryCriteria;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriteria;
+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);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.criteria())) {
+                if (idx == null) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else if (fldsCnt < idxFieldsCnt) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else continue;
+
+                // Best match. Index query criteria matches full index.
+                if (idxQryDesc.criteria().fields().size() == idxDef.indexKeyDefinitions().size())
+                    break;
+            }
+        }
+
+        return idx;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria) {
+        if (criteria.fields().size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.fields().get(i)))
+                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, IndexQueryCriteria 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, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        if (criteria instanceof RangeIndexQueryCriteria)
+            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) 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, RangeIndexQueryCriteria 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<RangeIndexQueryCriteria.RangeCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            String f = criteria.fields().get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriteria.RangeCriterion c = criteria.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 (match(get(), lower, 1) || match(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 match(IndexRow row, IndexRow boundary, int boundarySign) throws IgniteCheckedException {

Review comment:
       Bad method name. Actually, it returns true if not matched, so perhaps `notMatch` will be better, or left name as `match` but reverse return value logic.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryAllTypesTest.java
##########
@@ -0,0 +1,509 @@
+/*
+ * 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.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+public class IndexQueryAllTypesTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache<Long, Person> cache;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Long, Person> ccfg = new CacheConfiguration<Long, Person>()
+            .setName(CACHE)
+            .setIndexedTypes(Long.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeWithNulls() {
+        Function<Integer, Person> persGen = i -> {
+            Integer val = i < CNT / 10 ? null : i;
+
+            return person("intNullId", val);
+        };
+
+        insertData(i -> i, persGen, CNT);
+
+        int pivot = CNT / 5;
+
+        // Should include nulls.
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("intNullId", pivot));
+
+        check(cache.query(qry), 0, CNT / 5, i -> i, persGen);
+
+        // Should exclude nulls.
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(gte("intNullId", 0));
+
+        check(cache.query(qry), CNT / 10, CNT, i -> i, persGen);
+
+        // Should return only nulls.
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("intNullId", 0));
+
+        check(cache.query(qry), 0, CNT / 10, i -> i, persGen);
+
+        // Should return only nulls.
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lte("intNullId", null));
+
+        check(cache.query(qry), 0, CNT / 10, i -> i, persGen);
+
+        // Should return all non nulls.
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(gt("intNullId", null));
+
+        check(cache.query(qry), CNT / 10, CNT, i -> i, persGen);

Review comment:
       Lets also add check for `gte null`

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());
+        }
+    }
+
+    /** Should fail as no data on nodes. */
+    @Test
+    public void testClientNodeReplicatedCache() throws Exception {
+        startGrid();
+
+        Ignite cln = startClientGrid(1);
+
+        IgniteCache cache = cln.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(cln, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        GridTestUtils.assertThrows(null, () -> cache.query(qry.setLocal(true)).getAll(),

Review comment:
       Do we have the same exception for such a case with other query types? Message text looks confusing.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
##########
@@ -2277,6 +2285,21 @@ public boolean belongsToTable(GridCacheContext cctx, String expCacheName, String
         return true;
     }
 
+    /**
+     * Get table name by specified cache and cache value class.
+     *
+     * @param cacheName Cache name.
+     * @param valCls Value class.
+     * @return Table name or null if there is no match.
+     */
+    public String tableName(String cacheName, Class<?> valCls) {

Review comment:
       Lets annotate with `@Nullable`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.IndexQueryCriteria;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriteria;
+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);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.criteria())) {
+                if (idx == null) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else if (fldsCnt < idxFieldsCnt) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else continue;
+
+                // Best match. Index query criteria matches full index.
+                if (idxQryDesc.criteria().fields().size() == idxDef.indexKeyDefinitions().size())
+                    break;
+            }
+        }
+
+        return idx;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria) {
+        if (criteria.fields().size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.fields().get(i)))
+                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, IndexQueryCriteria 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, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        if (criteria instanceof RangeIndexQueryCriteria)
+            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) 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, RangeIndexQueryCriteria 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<RangeIndexQueryCriteria.RangeCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            String f = criteria.fields().get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriteria.RangeCriterion c = criteria.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 (match(get(), lower, 1) || match(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 match(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++) {
+                    RangeIndexQueryCriteria.RangeCriterion 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;
+
+                        return rowCmp.compareKey(o1.get(), o2.get(), 0);
+
+                    } catch (IgniteCheckedException e) {
+                        throw new IgniteException(e);
+                    }
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() throws IgniteCheckedException {
+            if (!returnFirst) {
+                for (int i = 0; i < cursors.length; i++) {
+                    if (!cursors[i].next()) {
+                        cursors[i] = null;
+                        cursorOff++;
+                    }
+                }
+
+                if (cursorOff == cursors.length)
+                    return false;
+
+                Arrays.sort(cursors, cursorComp);
+
+                returnFirst = true;
+

Review comment:
       Redundant NL

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TABLE = "TEST_CACHE_TABLE";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    private Ignite crd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration<>()
+            .setName(CACHE);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        prepareTable();
+
+        IgniteCache tableCache = crd.cache(CACHE_TABLE);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, DESC_ID_IDX)
+            .setCriteria(lt("descId", Integer.MAX_VALUE));
+
+        assertTrue(tableCache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testRangeQueries() {
+        prepareTable();
+
+        insertData();
+
+        int pivot = new Random().nextInt(CNT);
+
+        IgniteCache tableCache = crd.cache(CACHE_TABLE);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, DESC_ID_IDX)
+            .setCriteria(lt("descId", pivot));
+
+        check(tableCache.query(qry), 0, pivot);
+
+        // Wrong fields in query.
+        GridTestUtils.assertThrowsAnyCause(null, () -> {
+            IndexQuery<Long, Person> wrongQry = new IndexQuery<Long, Person>(Person.class, DESC_ID_IDX)
+                .setCriteria(lt("id", Integer.MAX_VALUE));
+
+            return cache.query(wrongQry).getAll();
+
+        }, IgniteCheckedException.class, "No index matches index query.");
+
+        // Wrong cache name.
+        GridTestUtils.assertThrowsAnyCause(null, () -> {
+            IndexQuery<Long, Person> wrongQry = new IndexQuery<Long, Person>(Person.class, DESC_ID_IDX)
+                .setCriteria(lt("descId", Integer.MAX_VALUE));
+
+            return cache.query(wrongQry).getAll();
+
+        }, IgniteCheckedException.class, "No index matches index query.");
+
+        // Wrong schema name.
+        GridTestUtils.assertThrowsAnyCause(null, () -> {

Review comment:
       Looks like all three `assertThrowsAnyCause` in this test checks the same case (the same cache, the same schema)

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java
##########
@@ -0,0 +1,598 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class MultifieldIndexQueryTest extends GridCommonAbstractTest {

Review comment:
       Let's also add a test where only one field of the multifield index used for filtering (for both INDEX and DESC_INDEX).

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class MultiTableIndexQuery extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>()
+            .setName("TEST_CACHE")
+            .setIndexedTypes(Long.class, Person.class, Long.class, SecondPerson.class)
+            .setQueryParallelism(4);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        QueryCursor<Cache.Entry<Long, Person>> cursor = cache.query(qry);
+
+        assertTrue(cursor.getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testLtQuery() {
+        insertData(cache);
+
+        int pivot = new Random().nextInt(CNT);
+
+        IndexQuery<Long, SecondPerson> secQry = new IndexQuery<Long, SecondPerson>(SecondPerson.class)
+            .setCriteria(lt("id", CNT + pivot));
+
+        checkSecondPerson(cache.query(secQry), CNT, CNT + pivot);
+    }
+
+    /** */
+    private void insertData(IgniteCache cache) {
+        for (int i = 0; i < CNT; i++) {
+            cache.put((long) i, new Person(i));
+            cache.put((long) CNT + i, new SecondPerson(CNT + i));
+        }
+    }
+
+    /** */
+    private void checkPerson(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {

Review comment:
       There are no usages of this method. 

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/MultifieldIndexQueryTest.java
##########
@@ -0,0 +1,598 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class MultifieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String INDEX = "TEST_IDX";
+
+    /** */
+    private static final String DESC_INDEX = "TEST_DESC_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    private Ignite ignite;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Collection<Object[]> testParams() {
+        return Arrays.asList(
+            new Object[] {1},
+            new Object[] {2});
+    }
+
+            /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        ignite = startGrids(nodesCnt);
+
+        cache = ignite.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>()
+            .setName("TEST_CACHE")
+            .setIndexedTypes(Long.class, Person.class)
+            .setQueryParallelism(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testQueryKeyPKIndex() {
+        insertData();
+
+        int pivot = new Random().nextInt(CNT);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class, "_key_PK")
+            .setCriteria(lt("_KEY", (long) pivot));
+
+        checkPerson(cache.query(qry), 0, pivot);
+    }
+
+    /** */
+    @Test
+    public void testEmptyCacheQuery() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE), lt("secId", Integer.MAX_VALUE));
+
+        QueryCursor<Cache.Entry<Long, Person>> cursor = cache.query(qry);
+
+        assertTrue(cursor.getAll().isEmpty());
+
+        // Check same query with specify index name.
+        qry = new IndexQuery<Long, Person>(Person.class, INDEX)
+            .setCriteria(lt("id", Integer.MAX_VALUE), lt("secId", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testCheckBoundaries() {
+        cache.put(1L, new Person(0, 1));
+        cache.put(2L, new Person(1, 0));
+        cache.put(3L, new Person(1, 1));
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(between("id", 0, 1), eq("secId", 1));
+
+        List<Cache.Entry<Long, Person>> result = cache.query(qry).getAll();
+
+        assertEquals(2, result.size());
+
+        result.sort(Comparator.comparingLong(Cache.Entry::getKey));
+
+        assertEquals(1L, (long) result.get(0).getKey());
+        assertEquals(3L, (long) result.get(1).getKey());
+
+        assertEquals(new Person(0, 1), result.get(0).getValue());
+        assertEquals(new Person(1, 1), result.get(1).getValue());
+    }
+
+    /** */
+    @Test
+    public void testLtQueryMultipleField() {
+        insertData();
+
+        int pivot = new Random().nextInt(CNT);
+
+        // Should return empty result for ID that less any inserted.
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", -1), lt("secId", pivot));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        // Should return all data for ID and SECID that greater any inserted.

Review comment:
       Don't understand this comment (and the following comments in this method). Why `greater` if the criteria are `lt`?

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class MultiTableIndexQuery extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>()
+            .setName("TEST_CACHE")
+            .setIndexedTypes(Long.class, Person.class, Long.class, SecondPerson.class)
+            .setQueryParallelism(4);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        QueryCursor<Cache.Entry<Long, Person>> cursor = cache.query(qry);
+
+        assertTrue(cursor.getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testLtQuery() {

Review comment:
       Let's also add a test with "_key_PK" index query for both tables.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TABLE = "TEST_CACHE_TABLE";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    private Ignite crd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration<>()

Review comment:
       Let's generify (also some other tests, `MultifieldIndexQueryTest` for example)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.IndexQueryCriteria;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriteria;
+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);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.criteria())) {
+                if (idx == null) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else if (fldsCnt < idxFieldsCnt) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else continue;
+
+                // Best match. Index query criteria matches full index.
+                if (idxQryDesc.criteria().fields().size() == idxDef.indexKeyDefinitions().size())
+                    break;
+            }
+        }
+
+        return idx;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria) {
+        if (criteria.fields().size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.fields().get(i)))
+                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, IndexQueryCriteria 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, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        if (criteria instanceof RangeIndexQueryCriteria)
+            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) 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, RangeIndexQueryCriteria 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<RangeIndexQueryCriteria.RangeCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            String f = criteria.fields().get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriteria.RangeCriterion c = criteria.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 (match(get(), lower, 1) || match(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 match(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++) {
+                    RangeIndexQueryCriteria.RangeCriterion 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;
+
+                        return rowCmp.compareKey(o1.get(), o2.get(), 0);
+

Review comment:
       Redundant NL

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.IndexQueryCriteria;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriteria;
+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);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.criteria())) {
+                if (idx == null) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else if (fldsCnt < idxFieldsCnt) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else continue;
+
+                // Best match. Index query criteria matches full index.
+                if (idxQryDesc.criteria().fields().size() == idxDef.indexKeyDefinitions().size())
+                    break;
+            }
+        }
+
+        return idx;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria) {
+        if (criteria.fields().size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.fields().get(i)))
+                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, IndexQueryCriteria 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, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        if (criteria instanceof RangeIndexQueryCriteria)
+            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) 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, RangeIndexQueryCriteria 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<RangeIndexQueryCriteria.RangeCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            String f = criteria.fields().get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriteria.RangeCriterion c = criteria.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 (match(get(), lower, 1) || match(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 match(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++) {
+                    RangeIndexQueryCriteria.RangeCriterion 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;
+
+                        return rowCmp.compareKey(o1.get(), o2.get(), 0);
+
+                    } catch (IgniteCheckedException e) {
+                        throw new IgniteException(e);
+                    }
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() throws IgniteCheckedException {
+            if (!returnFirst) {
+                for (int i = 0; i < cursors.length; i++) {
+                    if (!cursors[i].next()) {
+                        cursors[i] = null;
+                        cursorOff++;
+                    }
+                }
+
+                if (cursorOff == cursors.length)
+                    return false;
+
+                Arrays.sort(cursors, cursorComp);
+
+                returnFirst = true;
+
+            } else {

Review comment:
       NL after `{`




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r640646154



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {

Review comment:
       Would you mind introducing a ConditionBuilder as a separate class?




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660603011



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());

Review comment:
       Ok, you are right

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());

Review comment:
       Ok, you are right, let's check that sum equals to `CNT / 2`

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TABLE = "TEST_CACHE_TABLE";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    private Ignite crd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration<>()

Review comment:
       I mean you need to use generics (`CacheConfiguration<Object, Object>` for example), IDEA shows a lot of warning if you use raw types.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TABLE = "TEST_CACHE_TABLE";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    private Ignite crd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration<>()

Review comment:
       I mean you need to use generics (`CacheConfiguration<Object, Object>` for example), IDEA shows a lot of warnings if you use raw types.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660638230



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/MultiTableIndexQuery.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class MultiTableIndexQuery extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>()
+            .setName("TEST_CACHE")
+            .setIndexedTypes(Long.class, Person.class, Long.class, SecondPerson.class)
+            .setQueryParallelism(4);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        QueryCursor<Cache.Entry<Long, Person>> cursor = cache.query(qry);
+
+        assertTrue(cursor.getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testLtQuery() {
+        insertData(cache);
+
+        int pivot = new Random().nextInt(CNT);
+
+        IndexQuery<Long, SecondPerson> secQry = new IndexQuery<Long, SecondPerson>(SecondPerson.class)
+            .setCriteria(lt("id", CNT + pivot));
+
+        checkSecondPerson(cache.query(secQry), CNT, CNT + pivot);
+    }
+
+    /** */
+    private void insertData(IgniteCache cache) {
+        for (int i = 0; i < CNT; i++) {
+            cache.put((long) i, new Person(i));
+            cache.put((long) CNT + i, new SecondPerson(CNT + i));
+        }
+    }
+
+    /** */
+    private void checkPerson(QueryCursor<Cache.Entry<Long, Person>> cursor, int left, int right) {

Review comment:
       Added tests for check Person table too.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r653963660



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/RangeIndexCondition.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Range index condition that applies to BPlusTree based indexes.
+ */
+public class RangeIndexCondition implements IndexCondition {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** List of condition fields that should match index fields. */
+    private final List<String> fields = new ArrayList<>();
+
+    /** List of condition fields that should match index fields. */
+    private final List<SingleFieldRangeCondition> fldConds = new ArrayList<>();
+
+    /** */
+    public RangeIndexCondition(String field, @Nullable Object lower, @Nullable Object upper, boolean lowIncl, boolean upIncl) {
+        fields.add(field);
+
+        fldConds.add(
+            new SingleFieldRangeCondition(lower, upper, lowIncl, upIncl));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<String> fields() {
+        return fields;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexCondition and(IndexCondition cond) {
+        A.ensure(cond instanceof RangeIndexCondition, "Expect a range condition for chaining.");
+
+        RangeIndexCondition rngCond = (RangeIndexCondition) cond;
+
+        for (int i = 0; i < rngCond.fields.size(); i++) {
+            String f = rngCond.fields.get(i);
+
+            A.ensure(!fields.contains(f), "Duplicated field in conditions: " + f + ".");

Review comment:
       Discussed privately, decided to provide opportunity to define multiple conditions on the same field and merge them with AND. Will be done in separate ticket. 

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {

Review comment:
       done.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+        cond.addCondition(field2, null, wrapNull(vals[0]));
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], null, wrapNull(vals[i + 1]));
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val) {
+        lt(field, val);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val, String field2, Object... vals) {
+        lt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+        cond.addCondition(field2, wrapNull(vals[0]), null);
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], wrapNull(vals[i + 1]), null);
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val) {
+        gt(field, val);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val, String field2, Object... vals) {
+        gt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(lower), wrapNull(upper));
+
+        cond.lowerInclusive(true);
+        cond.upperInclusive(true);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper, String field2, Object... vals) {

Review comment:
       @tledkov-gridgain  Implemented new API, that covers this case.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696420722



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);

Review comment:
       What will if we pass criteria on columns A,B, but there is only index on column A.
   Guess, an out-of-bound exception will be thrown here.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685860079



##########
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:
       Got it. 
   Assume, there is a composite index on (A,B).
   So, we need to scan contiguous range on A (top level) and filter out B (lower levels).
   
   As I understand, idx.find(...) already returns values that satisfy condition on A, and (as you wrote) don't filter out conditions on B. 
   Why we need additional checks on A? Is this because we may need to filter out rows in case of exclusive bounds?
   If so, SortedIndex interface must specify whether bounds are inclusive or exclusive.
   
   Also, the rows that returned from the Cursor are materialized, even if they will be filtered out.
   It looks ok for now, but let's create a ticket for improvement.
   At first glance, IndexKey might have inclusive/exclusive flag, to avoid Row materialization if range has exclusive bounds. And we can think about passing a filter.
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660516364



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
##########
@@ -377,6 +377,28 @@ public IndexesRebuildTask idxRebuild() {
         }
     }
 
+    /**
+     * Returns index for specified name.
+     *
+     * @param idxName Index name.
+     * @return Index for specified index name.
+     */
+    public Index index(IndexName idxName) {
+        ddlLock.readLock().lock();
+
+        try {
+            Map<String, Index> idxs = cacheToIdx.get(idxName.cacheName());
+
+            if (idxs == null)
+                return null;
+
+            return idxs.get(idxName.fullName());
+
+        } finally {

Review comment:
       Also changed everywhere within this file.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.IndexQueryCriteria;
+import org.apache.ignite.cache.query.IndexQueryCriteriaBuilder;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriteria;
+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);
+
+        Index idx = null;
+        int idxFieldsCnt = 0;
+
+        for (Index i: idxs) {
+            IndexDefinition idxDef = idxProc.indexDefinition(i.id());
+
+            if (!tableName.equals(idxDef.idxName().tableName()))
+                continue;
+
+            int fldsCnt = idxDef.indexKeyDefinitions().size();
+
+            if (checkIndex(idxDef, idxQryDesc.criteria())) {
+                if (idx == null) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else if (fldsCnt < idxFieldsCnt) {
+                    idx = i;
+                    idxFieldsCnt = fldsCnt;
+                }
+                else continue;
+
+                // Best match. Index query criteria matches full index.
+                if (idxQryDesc.criteria().fields().size() == idxDef.indexKeyDefinitions().size())
+                    break;
+            }
+        }
+
+        return idx;
+    }
+
+    /** Checks that specified index matches index query criteria. */
+    private boolean checkIndex(IndexDefinition idxDef, IndexQueryCriteria criteria) {
+        if (criteria.fields().size() > idxDef.indexKeyDefinitions().size())
+            return false;
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            if (!idxDef.indexKeyDefinitions().get(i).name().equalsIgnoreCase(criteria.fields().get(i)))
+                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, IndexQueryCriteria 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, IndexQueryCriteria criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        if (criteria instanceof RangeIndexQueryCriteria)
+            return treeIndexRange((InlineIndex) idx, (RangeIndexQueryCriteria) 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, RangeIndexQueryCriteria 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<RangeIndexQueryCriteria.RangeCriterion> treeCriteria = new ArrayList<>();
+
+        for (int i = 0; i < criteria.fields().size(); i++) {
+            String f = criteria.fields().get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(f))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            RangeIndexQueryCriteria.RangeCriterion c = criteria.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 (match(get(), lower, 1) || match(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 match(IndexRow row, IndexRow boundary, int boundarySign) throws IgniteCheckedException {

Review comment:
       Rename to `exclude`

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());

Review comment:
       Don't understand why count of backups should affect result size? Actually we return data from primary nodes only.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());
+        }
+    }
+
+    /** Should fail as no data on nodes. */
+    @Test
+    public void testClientNodeReplicatedCache() throws Exception {
+        startGrid();
+
+        Ignite cln = startClientGrid(1);
+
+        IgniteCache cache = cln.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(cln, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        GridTestUtils.assertThrows(null, () -> cache.query(qry.setLocal(true)).getAll(),

Review comment:
       Yes, we have the same behavior for TextQuery. 

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryQueryEntityTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TBL_NAME = "TEST_CACHE_TBL_NAME";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache<Long, Person> cache;
+
+    /** */
+    private IgniteCache<Long, Person> cacheTblName;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+        cacheTblName = crd.cache(CACHE_TBL_NAME);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        QueryIndex idIdx = new QueryIndex("id", true, ID_IDX);
+        QueryIndex descIdIdx = new QueryIndex("descId", false, DESC_ID_IDX);
+
+        QueryEntity e = new QueryEntity(Long.class.getName(), Person.class.getName())
+            .setFields(new LinkedHashMap<>(
+                F.asMap("id", Integer.class.getName(), "descId", Integer.class.getName()))
+            )
+            .setIndexes(Arrays.asList(idIdx, descIdIdx));
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setQueryEntities(Collections.singletonList(e));
+
+        QueryEntity entTableName = new QueryEntity(e);
+        entTableName.setTableName(TABLE);
+
+        CacheConfiguration<?, ?> ccfg2 = new CacheConfiguration<>()
+            .setName(CACHE_TBL_NAME)
+            .setQueryEntities(Collections.singletonList(entTableName));
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("descId", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)

Review comment:
       Actually don't need it. Removed.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQuerySqlIndexTest.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQuerySqlIndexTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TABLE = "TEST_CACHE_TABLE";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache cache;
+
+    /** */
+    private Ignite crd;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration<>()

Review comment:
       How does it help? All tests (except 2) have different `getConfiguration()`, the only common thing is a cache name. But do we really need a separate class for that? Also flat structure make tests more convenient for debug. 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660563894



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryLocalTest.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryLocalTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** Should return full data. */
+    @Test
+    public void testServerNodeReplicatedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertEquals(CNT / 2, result.size());
+        }
+    }
+
+    /** Should return part of data only. */
+    @Test
+    public void testServerNodePartitionedCache() throws Exception {
+        Ignite crd = startGrids(4);
+
+        IgniteCache cache = crd.getOrCreateCache(ccfg(CacheMode.PARTITIONED));
+
+        insertData(crd, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        for (int i = 0; i < 4; i++) {
+            cache = grid(i).cache(CACHE);
+
+            List result = cache.query(qry.setLocal(true)).getAll();
+
+            assertTrue(CNT / 2 > result.size());
+        }
+    }
+
+    /** Should fail as no data on nodes. */
+    @Test
+    public void testClientNodeReplicatedCache() throws Exception {
+        startGrid();
+
+        Ignite cln = startClientGrid(1);
+
+        IgniteCache cache = cln.getOrCreateCache(ccfg(CacheMode.REPLICATED));
+
+        insertData(cln, cache);
+
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", CNT / 2));
+
+        GridTestUtils.assertThrows(null, () -> cache.query(qry.setLocal(true)).getAll(),

Review comment:
       Yes, we have the same behavior for TextQuery. 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685836840



##########
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:
       Those nulls is not about rows but about cursors (if some cursor is NULL then we compare it this way). But actually this check is useless as there cant be nulls at all. Will remove this.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r708203541



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       It's possible. There is no magic constant anymore.
   ```
    private boolean rowIsOutOfRange(IndexRow row, IndexRow low, IndexRow high) throws IgniteCheckedException {
           // Unbounded search, include all.
           if (low == null && high == null)
               return true;
   
           int criteriaKeysCnt = treeCriteria.size();
   
           for (int i = 0; i < criteriaKeysCnt; i++) {
               RangeIndexQueryCriterion c = treeCriteria.get(i);
   
               boolean descOrder = hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC;
   
               if (low != null && low.key(i) != null) {
                   int cmp = rowCmp.compareKey(row, low, i);
   
                   if (cmp == 0) {
                       if (!c.lowerIncl())
                           return true;     // Exclude if field equals boundary field and criteria is excluding.
                   }
                   // Cmp <> 0.
                   else if ((cmp < 0) ^ descOrder)
                       return true;
               }
   
               if (high != null && low.high(i) != null) {
                   int cmp = rowCmp.compareKey(row, high, i);
              
                   if (cmp == 0) {
                       if (!c.upperIncl())
                           return true;    // Exclude if field equals boundary field and criteria is excluding.
                   }
                   // Cmp != 0 here.
                   else if ((cmp > 0) ^ descOrder)
                       return true;
               }
           }
   
           return false;
       }
   
   ```




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696461174



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       Alternatives are:
   1. Additional storage for initial cursors.
   2. Store prefetched next value with additional `head` variable. (it breaks the GridCursor contract `next()` before)




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642650320



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       I'm agree that it's better to directly use index name. But currently name of index is an optional parameter. No QueryIndex, no QuerySqlField requires index name. So I believe we can query index by the same rules as we create it - just value class and list of fields. it should be enough.
   
   > Duplicating optimizer logic is a bad idea. It may be surprising for the user to get two different indexes in the SQL optimizer and IndexQuery for the same set of conditions
   
   IndexQuery doesn't promise optimize query. Even more it declares that IndexQuery doesn't waste time on planning, optimization, etc. It just get index that best match for condition and run a query on it. So for cases when optimization is required user has to specify index name.
   
   So I think we should use both ways of index definition.
   




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r708203541



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       It's possible. There is no magic constant anymore.
   ```
    private boolean include(IndexRow row, IndexRow low, IndexRow high) throws IgniteCheckedException {
           // Unbounded search, include all.
           if (low == null && high == null)
               return true;
   
           int criteriaKeysCnt = treeCriteria.size();
   
           for (int i = 0; i < criteriaKeysCnt; i++) {
               RangeIndexQueryCriterion c = treeCriteria.get(i);
   
               boolean descOrder = hnd.indexKeyDefinitions().get(i).order().sortOrder() == DESC;
   
               if (low != null && low.key(i) != null) {
                   int cmp = rowCmp.compareKey(row, low, i);
   
                   // Exclude if field equals boundary field and criteria is excluding.
                   if (cmp == 0) {
                       if (!c.lowerIncl())
                           return false;
                   }
                   // Cmp <> 0.
                   else if ((cmp < 0) ^ descOrder)
                       return false;
               }
   
               if (high != null && low.high(i) != null) {
                   int cmp = rowCmp.compareKey(row, high, i);
   
                   // Exclude if field equals boundary field and criteria is excluding.
                   if (cmp == 0) {
                       if (!c.upperIncl())
                           return false;
                   }
                   // Cmp != 0 here.
                   else if ((cmp > 0) ^ descOrder)
                       return false;
               }
           }
   
           return true;
       }
   
   ```




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642647280



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+        cond.addCondition(field2, null, wrapNull(vals[0]));
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], null, wrapNull(vals[i + 1]));
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val) {
+        lt(field, val);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val, String field2, Object... vals) {
+        lt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+        cond.addCondition(field2, wrapNull(vals[0]), null);
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], wrapNull(vals[i + 1]), null);
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val) {
+        gt(field, val);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val, String field2, Object... vals) {
+        gt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(lower), wrapNull(upper));
+
+        cond.lowerInclusive(true);
+        cond.upperInclusive(true);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper, String field2, Object... vals) {

Review comment:
       I'm not sure that chaining is good approach. Code looks clearly but actually it's not true. For example:
   ```
   IndexQuery.forIndex()
       .lt("fieldA", 1)
       .lt("fieldB", 2)
   ```
   
   There has to be a complex validation that different chain parts should be consistent:
   1. No gt() and lt() together on the same query;
   2. From other side it's OK to combine between() with other range methods;
   3. That max() or min() methods are the only methods.
   
   Also Repeating multiple times "lt" declaration is weird. It's enough to declare condition once, then we need only provide list of fields. 
   
   I think that alternative can be introducing a IndexField class that is argument of all index operations (varags too). 
   
   ```
   IndexQuery().forIndex()
       .lt(field("fieldA", 1))
       .pageSize(1024);
   
   IndexQuery().forIndex()
       .lt(
           field("fieldA", 1),
           field("fieldB", 2))
       .pageSize(1024);
   
   IndexQuery().forIndex()
       .between(
           field("fieldA", 1, 2),
           field("fieldB", 2, 3))
       .pageSize(1024);
   
   static IndexFieldCondition field(String field, Object val, Object? val2) { return new IndexFieldCondition(field, val, val2); }
   
   ```
   
   In this case it a little bit verbose for single field index, but looks OK for multiple fields index. 
   
   WDYT?
   




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r643004142



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {

Review comment:
       @AMashenkov hi! I think it is a good idea. I'll do it, but at first we should discuss comment @tledkov-gridgain about ugly between method. Solution will affect how the build looks like.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685903120



##########
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:
       You pass Row (as result of get()) and Condition (search row) to exclude() method, but they have same type (IndexRow)
   but comparison logic that differs from what happens inside the Index.
   So, 'boundary' param name looks confusing.




-- 
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



[GitHub] [ignite] timoninmaxim commented on pull request #9118: IGNITE-14699 Add IndexQuery API.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#issuecomment-899405918


   @AMashenkov hi! I fixed merge conflict. Now, this PR is available for review again.


-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696432170



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       It would be much cleaner to have a method with signature `boolean exclude(row, lower, upper)` 
   or even `boolean rowInRange(row, lower, upper)` 
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685800876



##########
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:
       Order of fields matters, as it's possible to create index (A, B) and (B, A). Also in case we have index (A, B) user can specify query for (B) only. It's illegal. User has to start with most significant field. So I think we should be restrictive there.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642557531



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       The logic of choose index by IndexQuery may confuse user in case multiple-filed index. I propose to think about QueryIndex as low-level API for index access. I guess public Index interface and find/scan methods will be more appropriate. 
   
   Let's disallow the creation of a IndexQuery without explicitly specifying the index.
   
   I'll explain my point of view.
   
   The choice of the index is the job of the SQL optimizer. The optimizer can use statistics and other metadata.
   
   Duplicating optimizer logic is a bad idea.
   It may be surprising for the user to get two different indexes in the SQL optimizer and IndexQuery for the same set of conditions.




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696483713



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       > Did we already swapped boundaries in treeIndexRange() ?
   
   In treeRange we use swap for user's defined criteria - it's required to correctly work with DESC indexes with mirrored criteria (gt() for DESC index will work unexpected for user). But it doesn't affect results of comparing. We need correctly work with DESC indexes while compare - consider additional -1 multiplier. InlineIndexTree does this for traversing part, but we also need do it for filter part.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642583342



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+        cond.addCondition(field2, null, wrapNull(vals[0]));
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], null, wrapNull(vals[i + 1]));
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val) {
+        lt(field, val);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val, String field2, Object... vals) {
+        lt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+        cond.addCondition(field2, wrapNull(vals[0]), null);
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], wrapNull(vals[i + 1]), null);
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val) {
+        gt(field, val);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val, String field2, Object... vals) {
+        gt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(lower), wrapNull(upper));
+
+        cond.lowerInclusive(true);
+        cond.upperInclusive(true);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper, String field2, Object... vals) {

Review comment:
       Looks ugly for me. Can it be relpaced by several call of between with accumulate fields? 




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r698549048



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       Added a wrapped method `rowIsOutOfRange` with clear javadoc.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       Simplified this code. Now we prepare fill the priority queue in constructor, and prepare `head` on `next()` step, and it doesn't violate the contract.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r660588619



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryQueryEntityTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+
+/** */
+public class IndexQueryQueryEntityTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String CACHE_TBL_NAME = "TEST_CACHE_TBL_NAME";
+
+    /** */
+    private static final String TABLE = "TEST_TABLE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private IgniteCache<Long, Person> cache;
+
+    /** */
+    private IgniteCache<Long, Person> cacheTblName;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignite crd = startGrids(4);
+
+        cache = crd.cache(CACHE);
+        cacheTblName = crd.cache(CACHE_TBL_NAME);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        QueryIndex idIdx = new QueryIndex("id", true, ID_IDX);
+        QueryIndex descIdIdx = new QueryIndex("descId", false, DESC_ID_IDX);
+
+        QueryEntity e = new QueryEntity(Long.class.getName(), Person.class.getName())
+            .setFields(new LinkedHashMap<>(
+                F.asMap("id", Integer.class.getName(), "descId", Integer.class.getName()))
+            )
+            .setIndexes(Arrays.asList(idIdx, descIdIdx));
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setQueryEntities(Collections.singletonList(e));
+
+        QueryEntity entTableName = new QueryEntity(e);
+        entTableName.setTableName(TABLE);
+
+        CacheConfiguration<?, ?> ccfg2 = new CacheConfiguration<>()
+            .setName(CACHE_TBL_NAME)
+            .setQueryEntities(Collections.singletonList(entTableName));
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testEmptyCache() {
+        IndexQuery<Long, Person> qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("descId", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)
+            .setCriteria(lt("id", Integer.MAX_VALUE));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Long, Person>(Person.class)

Review comment:
       Actually don't need it. Removed.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696459761



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       We can't compare head of cursors before user invoked `next()`. So then we need a storage for cursors. To avoid additional class-level structure for this, I propose to store cursors within the same storage - PriorityQueue. After first `next()` we re-fill storage. 
   
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685408254



##########
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:
       But RangeIndexQueryCriterion already has the flags. 1Why you will need array of flags? 
   RangeIndexQueryCriterion has all the information to answer whether Nulls should be included into the result or not. 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r685963686



##########
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:
       > Is this because we may need to filter out rows in case of exclusive bounds
   
   Yes, this is a reason. Unfortunately, currently our tree structure doesn't support exclusive bounds. It looks like a major change in tree. I think it should be done in separate ticket.
   
   > Also, the rows that returned from the Cursor are materialized
   
   There is already a ticket for that [IGNITE-14945](https://issues.apache.org/jira/browse/IGNITE-14945). It filters basing on inlined values. So it will speed up filtering much.  
   
   




-- 
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



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

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696435957



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -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;
+            }
+        };
+    }
+
+    /**
+     * @param isNull {@code true} if user explicitly set {@code null} with a query argument.
+     */
+    private IndexKey key(Object val, boolean isNull, IndexKeyDefinition def, IndexKeyTypeSettings settings, CacheObjectContext coctx) {
+        IndexKey key = null;
+
+        if (val != null || isNull)
+            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 PriorityQueue<GridCursor<IndexRow>> cursors;
+
+        /** Whether returns first value for user. */
+        private boolean returnFirst;
+
+        /** Comparator to compare index rows. */
+        private final Comparator<GridCursor<IndexRow>> cursorComp;
+
+        /** */
+        SegmentedIndexCursor(GridCursor<IndexRow>[] cursors, IndexRowComparator rowCmp) {
+            cursorComp = new Comparator<GridCursor<IndexRow>>() {
+                @Override public int compare(GridCursor<IndexRow> o1, GridCursor<IndexRow> o2) {
+                    try {
+                        if (!returnFirst)
+                            return -1;

Review comment:
       This looks like a dirty hack.
   Why do we need this flag? Let's drop it.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r642583342



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/IndexQuery.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+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 condition.
+ */
+@IgniteExperimental
+public class IndexQuery<K, V> extends Query<Cache.Entry<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Object to mark a boundary if {@code null} is specified. */
+    private static final Object NULL = new Null();
+
+    /** Index condition describes index query clause. */
+    private IndexCondition idxCond;
+
+    /** Cache Value class. Describes a table within a cache that runs a query. */
+    private final String valCls;
+
+    /** Optional index name. Find index by fields in condition. */
+    private final @Nullable String idxName;
+
+    /** Optional schema name. User has to specify schema to run query over an index created with SQL. */
+    private final @Nullable String schema;
+
+    /** */
+    private IndexQuery(String valCls, @Nullable String idxName, @Nullable String schema) {
+        this.valCls = valCls;
+        this.idxName = idxName;
+        this.schema = schema;
+    }
+
+    /**
+     * Specify index with cache value class. Ignite checks all indexes to find best match by
+     * {@link #valCls} and {@link IndexCondition#fields()}.
+     */
+    public static <K, V> IndexQuery<K, V> forType(Class<V> valCls) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), null, null);
+    }
+
+    /**
+     * Specify index with cache value class and index name.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, null);
+    }
+
+    /**
+     * Specify index with cache value class, index name and schema name.
+     * Note that schema is required parameter for indexes created with the "CREATE INDEX" SQL-clause.
+     */
+    public static <K, V> IndexQuery<K, V> forIndex(Class<V> valCls, String idxName, String schema) {
+        A.notNull(valCls, "valCls");
+
+        return new IndexQuery<>(valCls.getName(), idxName, schema);
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then. */
+    public IndexQuery<K, V> lt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, null, wrapNull(val));
+        cond.addCondition(field2, null, wrapNull(vals[0]));
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], null, wrapNull(vals[i + 1]));
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val) {
+        lt(field, val);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Less Then or Equal. */
+    public IndexQuery<K, V> lte(String field, Object val, String field2, Object... vals) {
+        lt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).upperInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then. */
+    public IndexQuery<K, V> gt(String field, Object val, String field2, Object... vals) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        A.notEmpty(vals, "vals");
+        A.ensure(vals.length % 2 == 1, "number of fields has to be equal number of values.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(val), null);
+        cond.addCondition(field2, wrapNull(vals[0]), null);
+
+        for (int i = 1; i + 1 < vals.length; i += 2) {
+            A.ensure(vals[i] instanceof String, "waited for field name but got " + vals[i]);
+
+            cond.addCondition((String) vals[i], wrapNull(vals[i + 1]), null);
+        }
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val) {
+        gt(field, val);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Greater Then or Equal. */
+    public IndexQuery<K, V> gte(String field, Object val, String field2, Object... vals) {
+        gt(field, val, field2, vals);
+
+        ((RangeIndexCondition) idxCond).lowerInclusive(true);
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper) {
+        A.ensure(idxCond == null, "The only index condition is supported.");
+
+        RangeIndexCondition cond = new RangeIndexCondition();
+
+        cond.addCondition(field, wrapNull(lower), wrapNull(upper));
+
+        cond.lowerInclusive(true);
+        cond.upperInclusive(true);
+
+        idxCond = cond;
+
+        return this;
+    }
+
+    /** Between. Lower and upper boundaries are inclusive. */
+    public IndexQuery<K, V> between(String field, Object lower, Object upper, String field2, Object... vals) {

Review comment:
       Looks ugly for me. Can it be relpaced by several call of the between() method with accumulate fields? 




-- 
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.

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



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

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r698549048



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.lang.reflect.Array;
+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 java.util.PriorityQueue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.IndexQuery;
+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);
+
+        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)
+            throw failIndexQuery("No table found: " + idxQryDesc.valCls(), cctx, idxQryDesc);
+
+        if (idxQryDesc.idxName() != null) {
+            Index idx = indexByName(cctx, idxQryDesc, tableName);
+
+            if (idx == null)
+                throw failIndexQuery("No index found: " + idxQryDesc.idxName(), cctx, idxQryDesc);
+
+            return idx;
+        }
+
+        Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+
+        if (idx == null)
+            throw failIndexQuery("No index matches query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** Get index by name, or return {@code null}. */
+    private Index indexByName(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) throws IgniteCheckedException {
+        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;
+
+        if (!checkIndex(idxProc.indexDefinition(idx.id()), idxQryDesc.criteria()))
+            throw failIndexQuery("Index " + idxQryDesc.idxName() + " doesn't match query", cctx, idxQryDesc);
+
+        return idx;
+    }
+
+    /** */
+    private IgniteCheckedException failIndexQuery(String msg, GridCacheContext cctx, IndexQueryDesc desc) {
+        return new IgniteCheckedException(
+            "Failed to parse IndexQuery. " + msg + ". Cache=" + cctx.name() + "; Query=" + desc);
+    }
+
+    /**
+     * Get index by list of fields to query, or return {@code null}.
+     *
+     * Note, it tries to find best index match: count of index fields should equal to count of index criteria fields.
+     */
+    private Index indexByCriteria(GridCacheContext cctx, IndexQueryDesc idxQryDesc, String tableName) {
+        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 an index query.
+     *
+     * @return Result cursor over 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());
+    }
+
+    /**
+     * Runs an index query for single {@code segment}.
+     *
+     * @return Result cursor over segment.
+     */
+    private GridCursor<IndexRow> query(int segment, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+        throws IgniteCheckedException {
+
+        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).cacheGroupContext().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++) {
+            RangeIndexQueryCriterion c = (RangeIndexQueryCriterion) criteria.get(i);
+
+            IndexKeyDefinition def = hnd.indexKeyDefinitions().get(i);
+
+            if (!def.name().equalsIgnoreCase(c.field()))
+                throw new IgniteCheckedException("Range query doesn't match index '" + idx.name() + "'");
+
+            // 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(), c.lowerNull(), def, hnd.indexKeyTypeSettings(), coctx);
+            IndexKey u = key(c.upper(), c.upperNull(), 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 = -cmp;

Review comment:
       Added a wrapped method `rowIsOutOfRange` with clear javadoc.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9118:
URL: https://github.com/apache/ignite/pull/9118#discussion_r696416566



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+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.internal.cache.query.IndexCondition;
+import org.apache.ignite.internal.cache.query.RangeIndexCondition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.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.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;
+
+/**
+ * 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.idxCond(), 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 = "_key_PK".equals(idxQryDesc.idxName()) ? "_key_PK" : idxQryDesc.idxName().toUpperCase();
+
+            String schema = idxQryDesc.schema() == null ? cctx.name() : idxQryDesc.schema();
+
+            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.idxCond()) ? idx : null;
+        }
+
+        // Try get index by list of fields to query.

Review comment:
       > No QueryIndex, no QuerySqlField requires index name
   They include SQL engine, optimizer and cost model under cover. 
   Choose the index by conditions - the work for optimizer.
   
   It looks like the order of the criteria should be the same as the order of the index fields.
   You really means that it is simple to understand for user than specify the index name?
   My point: index name must be specified by the user. IndexAPI must not find appropriate index.
   Lets discuss it on the devlist to get more opinions.




-- 
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