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/09/21 15:45:18 UTC

[GitHub] [ignite] timoninmaxim opened a new pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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


   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.

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

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



[GitHub] [ignite] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r722097495



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {
+                    RangeIndexQueryCriterion c1 = (RangeIndexQueryCriterion)criterion(i, fldName, pair.get1(), pair.get2());
+                    RangeIndexQueryCriterion c2 = (RangeIndexQueryCriterion)criterion(j, fldName, pair.get2(), pair.get1());
+
+                    cc.add(new T2<>(c1, c2));
+                }
+            }
+        });
+
+        cc.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        checkEqualsCriteria(lower, upper, 0, CNT);
+        checkEqualsCriteria(lower, upper, CNT, 0);
+        checkEqualsCriteria(upper, lower, 0, CNT);
+        checkEqualsCriteria(upper, lower, CNT, 0);
+    }
+
+    /** */
+    private void checkEqualsCriteria(int eq1, int eq2, int from, int to) {
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, eq1), eq(fldName, eq2), between(fldName, from, to));
+
+        assertTrue(qry.getCriteria().toString(), cache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testCommonBoundary() {

Review comment:
       test check nothing common 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] timoninmaxim commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       > BTW, is it possible that onlyOne.indexSearchRow()
   
   Yes, it's case of finding range in index rows by search row. 




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r722070347



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {

Review comment:
       Why not just call `compareKey` from `compareRow` when both.indexSearchRow() (BTW, is it possible that onlyOne.indexSearchRow()?)
   
   I see we duplicate the fillowing 
   ```
           int ltype = DataType.getTypeFromClass(left.key().getClass());
           int rtype = DataType.getTypeFromClass(right.key().getClass());
   
           int c = compareValues(wrap(left.key(), ltype), wrap(right.key(), rtype));
   
           return Integer.signum(c);
   ```
   and wish this can be fixed )




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r722086957



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       Why not just call `compareKey` from `compareRow` when both.indexSearchRow() (BTW, is it possible that onlyOne.indexSearchRow()?)
   
   I see we duplicate the following
   ```
           int ltype = DataType.getTypeFromClass(left.key().getClass());
           int rtype = DataType.getTypeFromClass(right.key().getClass());
   
           int c = compareValues(wrap(left.key(), ltype), wrap(right.key(), rtype));
   
           return Integer.signum(c);
   ```
   and wish this can be fixed )




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {

Review comment:
       I think no. Check the method `RepeatedFieldIndexQueryTest#criterion`. It contains all numbers in this range. In case using constant for 6 we should create constants for every method there. I think it's overkill.




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r729935337



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/IndexQueryFailoverTest.java
##########
@@ -174,11 +170,29 @@ public void testQueryWrongQuery() {
                 return cache.query(qry).getAll();
             },
             IgniteCheckedException.class, errMsg);
+
+        Stream.of(
+            Arrays.asList(lt("id", 100), gt("id", 101)),
+            Arrays.asList(eq("id", 100), eq("id", 101)),
+            Arrays.asList(eq("id", 101), eq("id", 100)),
+            Arrays.asList(eq("id", 101), between("id", 19, 40)),
+            Arrays.asList(between("id", 432, 40))
+        ).forEach(crit -> {
+            String msg = "Criterion is invalid: lower boundary is greater than upper";

Review comment:
       Message does not look to be proper for case `eq("id", 100), eq("id", 101)`

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<IndexQueryCriterion> criteria1 = criteria(fldName, lower, upper);
+        List<IndexQueryCriterion> criteria2 = criteria(fldName, upper, lower);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> checks = new ArrayList<>();
+
+        for (int i = 0; i < criteria1.size(); i++) {
+            for (int j = 0; j < criteria2.size(); j++) {
+                checks.add(new T2<>(
+                    (RangeIndexQueryCriterion)criteria1.get(i),
+                    (RangeIndexQueryCriterion)criteria2.get(j)));
+            }
+
+        }
+
+        checks.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMergeMultipleCriteriaForSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(gt(fldName, lower), gt(fldName, lower - 1), gt(fldName, lower - 2),
+                lt(fldName, upper), lt(fldName, upper + 1), lt(fldName, upper + 2));
+
+        check(null, cache.query(qry), lower + 1, upper);
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        checkEqualsCriteria(lower, upper, 0, CNT);
+        checkEqualsCriteria(lower, upper, CNT, 0);
+        checkEqualsCriteria(upper, lower, 0, CNT);
+        checkEqualsCriteria(upper, lower, CNT, 0);
+    }
+
+    /** */
+    private void checkEqualsCriteria(int eq1, int eq2, int from, int to) {
+        GridTestUtils.assertThrows(null, () -> {
+            IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+                .setCriteria(eq(fldName, eq1), eq(fldName, eq2), between(fldName, from, to));

Review comment:
       any reason for `between` when we checking equals?
   we MAY have special tests with additional `between`, but `eq-and-eq` test is required (`between` may hide some problem here)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -128,15 +123,37 @@ public IndexQueryProcessor(IndexProcessor idxProc) {
         };
     }
 
-    /** Get index to run query by specified description. */
-    private Index index(GridCacheContext<?, ?> cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
-        String tableName = cctx.kernalContext().query().tableName(cctx.name(), idxQryDesc.valType());
+    /**
+     * Get index to run query by specified description. Validates that criteria fields matches a prefix of fields
+     * of found index.
+     */
+    private Index getAndValidateIndex(GridCacheContext<?, ?> cctx, IndexQueryDesc idxQryDesc) throws IgniteCheckedException {
+        final String tableName = cctx.kernalContext().query().tableName(cctx.name(), idxQryDesc.valType());
 
         if (tableName == null)
             throw failIndexQuery("No table found for type: " + idxQryDesc.valType(), null, idxQryDesc);
 
-        if (idxQryDesc.idxName() == null && !F.isEmpty(idxQryDesc.criteria())) {
-            Index idx = indexByCriteria(cctx, idxQryDesc, tableName);
+        // Collect both fields (original and normalized).
+        Map<String, String> critFlds;
+
+        if (!F.isEmpty(idxQryDesc.criteria())) {
+            critFlds = idxQryDesc.criteria().stream()
+                .map(IndexQueryCriterion::field)
+                .flatMap(f -> {
+                    String norm = QueryUtils.normalizeObjectName(f, false);
+
+                    if (f.equals(norm))
+                        return Stream.of(new T2<>(f, f));
+                    else
+                        return Stream.of(new T2<>(f, norm), new T2<>(norm, f));
+                })
+                .collect(Collectors.toMap(IgniteBiTuple::get1, IgniteBiTuple::get2, (l, r) -> l));
+        }
+        else
+            critFlds = Collections.emptyMap();
+
+        if (idxQryDesc.idxName() == null && !critFlds.isEmpty()) {
+            Index idx = indexByCriteria(cctx, critFlds, tableName);
 
             if (idx == null)
                 throw failIndexQuery("No index found for criteria", null, idxQryDesc);

Review comment:
       could be inlined to the `indexByCriteria`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexQueryProcessor.java
##########
@@ -243,83 +254,176 @@ private IgniteCheckedException failIndexQuery(String msg, IndexDefinition idxDef
         return new IgniteCheckedException(exMsg + " Query=" + desc);
     }
 
-    /** Checks that specified index matches index query criteria. */
-    private List<IndexQueryCriterion> alignCriteriaWithIndex(IndexDefinition idxDef, IndexQueryDesc idxQryDesc)
-        throws IgniteCheckedException {
-        if (idxQryDesc.criteria().size() > idxDef.indexKeyDefinitions().size())
-            throw failIndexQueryCriteria(idxDef, idxQryDesc);
+    /** Merges multiple criteria for the same field into single criterion. */
+    private Map<String, RangeIndexQueryCriterion> mergeIndexQueryCriteria(
+        InlineIndexImpl idx,
+        SortedIndexDefinition idxDef,
+        IndexQueryDesc idxQryDesc
+    ) throws IgniteCheckedException {
+        Map<String, RangeIndexQueryCriterion> mergedCriteria = new HashMap<>();
 
-        Map<String, IndexQueryCriterion> critFlds = new HashMap<>();
-        Map<String, IndexQueryCriterion> normCritFlds = new HashMap<>();
+        Map<String, IndexKeyDefinition> idxFlds = idxDef.indexKeyDefinitions();
+        IndexKeyTypeSettings keyTypeSettings = idx.segment(0).rowHandler().indexKeyTypeSettings();
+        CacheObjectContext coctx = idx.segment(0).cacheGroupContext().cacheObjectContext();
 
-        // We need check both original and normalized field names.
-        idxQryDesc.criteria().forEach(c -> {
-            critFlds.put(c.field(), c);
-            normCritFlds.put(c.field().toUpperCase(), c);
-        });
+        IndexRowComparator keyCmp = idxDef.rowComparator();
 
-        List<IndexQueryCriterion> aligned = new ArrayList<>();
+        for (IndexQueryCriterion c: idxQryDesc.criteria()) {
+            RangeIndexQueryCriterion crit = (RangeIndexQueryCriterion) c;
 
-        // Checks that users criteria matches a prefix subset of index fields.
-        for (int i = 0; i < idxQryDesc.criteria().size(); i++) {
-            String idxFld = idxDef.indexKeyDefinitions().get(i).name();
+            String fldName = idxFlds.containsKey(crit.field()) ? crit.field()
+                : QueryUtils.normalizeObjectName(crit.field(), false);
+
+            IndexKeyDefinition keyDef = idxFlds.get(fldName);
+
+            if (keyDef == null)
+                throw failIndexQuery("Index doesn't match criteria", idxDef, idxQryDesc);
+
+            IndexKey l = key(crit.lower(), crit.lowerNull(), keyDef, keyTypeSettings, coctx);
+            IndexKey u = key(crit.upper(), crit.upperNull(), keyDef, keyTypeSettings, coctx);
+
+            boolean lowIncl = crit.lowerIncl();
+            boolean upIncl = crit.upperIncl();
+
+            boolean lowNull = crit.lowerNull();
+            boolean upNull = crit.upperNull();
 
-            IndexQueryCriterion c = normCritFlds.remove(idxFld);
+            if (mergedCriteria.containsKey(fldName)) {
+                RangeIndexQueryCriterion prev = mergedCriteria.get(fldName);
 
-            if (c == null) {
-                // Check this field is escaped.
-                c = critFlds.remove(idxFld);
+                int lowCmp = 0;
 
-                if (c == null)
-                    throw failIndexQueryCriteria(idxDef, idxQryDesc);
+                // Use previous lower boudary, as it's greater than the current.
+                if (l == null || (prev.lower() != null && (lowCmp = keyCmp.compareKey((IndexKey)prev.lower(), l)) >= 0)) {
+                    l = (IndexKey)prev.lower();
+                    lowIncl = lowCmp != 0 ? prev.lowerIncl() : prev.lowerIncl() ? lowIncl : prev.lowerIncl();
+                    lowNull = prev.lowerNull();
+                }
+
+                int upCmp = 0;
+
+                // Use previous upper boudary, as it's less than the current.
+                if (u == null || (prev.upper() != null && (upCmp = keyCmp.compareKey((IndexKey)prev.upper(), u)) <= 0)) {
+                    u = (IndexKey)prev.upper();
+                    upIncl = upCmp != 0 ? prev.upperIncl() : prev.upperIncl() ? upIncl : prev.upperIncl();
+                    upNull = prev.upperNull();
+                }
             }
-            else
-                critFlds.remove(c.field());
 
-            aligned.add(c);
+            if (l != null && u != null && idxDef.rowComparator().compareKey(l, u) > 0)
+                throw failIndexQuery("Criterion is invalid: lower boundary is greater than upper. Field=" + fldName +
+                    ", lower=" + l.key() + ", upper=" + u.key(), idxDef, idxQryDesc);
+
+            RangeIndexQueryCriterion idxKeyCrit = new RangeIndexQueryCriterion(fldName, l, u);
+            idxKeyCrit.lowerIncl(lowIncl);
+            idxKeyCrit.upperIncl(upIncl);
+            idxKeyCrit.lowerNull(lowNull);
+            idxKeyCrit.upperNull(upNull);
+
+            mergedCriteria.put(fldName, idxKeyCrit);
         }
 
-        if (!critFlds.isEmpty())
-            throw failIndexQueryCriteria(idxDef, idxQryDesc);
+        return mergedCriteria;
+    }
 
-        return aligned;
+    /** Checks that specified index matches index query criteria. */
+    private IndexRangeQuery alignCriteriaWithIndex(
+        InlineIndexImpl idx,
+        Map<String, RangeIndexQueryCriterion> criteria,
+        IndexDefinition idxDef
+    ) {
+        // Size of bounds array has to be equal to count of indexed fields.
+        IndexKey[] lowerBounds = new IndexKey[idxDef.indexKeyDefinitions().size()];
+        IndexKey[] upperBounds = new IndexKey[idxDef.indexKeyDefinitions().size()];
+
+        boolean lowerAllNulls = true;
+        boolean upperAllNulls = true;
+
+        IndexRangeQuery qry = new IndexRangeQuery(criteria.size());
+
+        // Checks that users criteria matches a prefix subset of index fields.
+        int i = 0;
+
+        for (Map.Entry<String, IndexKeyDefinition> keyDef: idxDef.indexKeyDefinitions().entrySet()) {
+            RangeIndexQueryCriterion criterion = criteria.remove(keyDef.getKey());
+
+            if (keyDef.getValue().order().sortOrder() == DESC)
+                criterion = criterion.swap();
+
+            qry.criteria[i] = criterion;
+
+            IndexKey l = (IndexKey) criterion.lower();
+            IndexKey u = (IndexKey) criterion.upper();
+
+            if (l != null)
+                lowerAllNulls = false;
+
+            if (u != null)
+                upperAllNulls = false;
+
+            lowerBounds[i] = l;
+            upperBounds[i++] = u;
+
+            if (criteria.isEmpty())
+                break;
+        }
+
+        InlineIndexRowHandler hnd = idx.segment(0).rowHandler();
+
+        qry.lower = lowerAllNulls ? null : new IndexSearchRowImpl(lowerBounds, hnd);
+        qry.upper = upperAllNulls ? null : new IndexSearchRowImpl(upperBounds, hnd);
+
+        return qry;
     }
 
     /**
      * Runs an index query.
      *
      * @return Result cursor over index segments.
      */
-    private GridCursor<IndexRow> query(GridCacheContext<?, ?> cctx, Index idx, List<IndexQueryCriterion> criteria, IndexQueryContext qryCtx)
+    private GridCursor<IndexRow> query(GridCacheContext<?, ?> cctx, Index idx, IndexQueryDesc idxQryDesc, IndexQueryContext qryCtx)
         throws IgniteCheckedException {
 
-        int segmentsCnt = cctx.isPartitioned() ? cctx.config().getQueryParallelism() : 1;
-
-        if (segmentsCnt == 1)
-            return query(0, idx, criteria, qryCtx);
-
-        final GridCursor<IndexRow>[] segmentCursors = new GridCursor[segmentsCnt];
+        if (F.isEmpty(idxQryDesc.criteria()) || idxQryDesc.criteria().get(0) instanceof RangeIndexQueryCriterion)
+            return querySortedIndex(cctx, (InlineIndexImpl) idx, idxQryDesc, qryCtx);
 
-        // Actually it just traverses BPlusTree to find boundaries. It's too fast to parallelize this.
-        for (int i = 0; i < segmentsCnt; i++)
-            segmentCursors[i] = query(i, idx, criteria, qryCtx);
+        IndexQueryCriterion c = idxQryDesc.criteria().get(0);

Review comment:
       `idxQryDesc.criteria().get(0)` duplicated across the method.




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r716812646



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       Looks like this method partially duplicates `H2RowComparator#compareKey`
   Should it use this method instead?




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       > Why not just call compareKey from compareRow when both.indexSearchRow()
   
   Because I don't have IndexRow in moment of merging user's criteria. It's just number of index keys of the same field. I should check every boundary of every field. And only after that I can create an IndexRow (that is a search row).




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       Hi, name of methods are little confusing. I've renamed them. Note, that `IndexRowComparator#compareRow` accepts IndexRow. In some cases we can't replace it with `compareKey` due to additional logic in this method.




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       > BTW, is it possible that onlyOne.indexSearchRow()
   
   Yes, it's case of finding range in index rows by search row (it's ordinary case for IndexQuery or SqlFieldsQuery). In this moment we compare user's search row with stored (non-search) index row.




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {

Review comment:
       Fixed, use predefined list instead of `switch`




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r721494432



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       Question is still open




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       > Why not just call compareKey from compareRow when both.indexSearchRow()
   
   Because I don't have IndexRow in moment of merging user's criteria. It's just number of index keys of the same field. I should check every boundary of every field. And only after that I can create an IndexRow.




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r721465853



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {

Review comment:
       should this be some const?

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {
+                    RangeIndexQueryCriterion c1 = (RangeIndexQueryCriterion)criterion(i, fldName, pair.get1(), pair.get2());
+                    RangeIndexQueryCriterion c2 = (RangeIndexQueryCriterion)criterion(j, fldName, pair.get2(), pair.get1());
+
+                    cc.add(new T2<>(c1, c2));
+                }
+            }
+        });
+
+        cc.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, upper), eq(fldName, lower), between(fldName, 0, CNT));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, lower), eq(fldName, upper), between(fldName, 0, CNT));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, lower), eq(fldName, upper), between(fldName, CNT, 0));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, lower), eq(fldName, upper), between(fldName, CNT, 0));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());

Review comment:
       It looks like we are able to have some method creates a query and checks it's produces an empty set.
   This will make the test smaller and more readable.
   And, possible we'll check more cases %) 

##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {
+                    RangeIndexQueryCriterion c1 = (RangeIndexQueryCriterion)criterion(i, fldName, pair.get1(), pair.get2());
+                    RangeIndexQueryCriterion c2 = (RangeIndexQueryCriterion)criterion(j, fldName, pair.get2(), pair.get1());
+
+                    cc.add(new T2<>(c1, c2));
+                }
+            }
+        });
+
+        cc.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, upper), eq(fldName, lower), between(fldName, 0, CNT));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, lower), eq(fldName, upper), between(fldName, 0, CNT));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, lower), eq(fldName, upper), between(fldName, CNT, 0));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, lower), eq(fldName, upper), between(fldName, CNT, 0));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testCommonBoundary() {
+        int upper, lower = upper = new Random().nextInt(CNT / 2);
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(lt(fldName, lower), gt(fldName, upper));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(lte(fldName, lower), gt(fldName, upper));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(lt(fldName, lower), gte(fldName, upper));
+
+        assertTrue(cache.query(qry).getAll().isEmpty());
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(lte(fldName, lower), gte(fldName, upper));
+
+        check(null, cache.query(qry), lower, upper + 1);
+
+        qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(between(fldName, 0, lower), between(fldName, upper, CNT));
+
+        check(null, cache.query(qry), lower, upper + 1);

Review comment:
       same here. every 4 lines can be presented by one.




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<IndexQueryCriterion> criteria1 = criteria(fldName, lower, upper);
+        List<IndexQueryCriterion> criteria2 = criteria(fldName, upper, lower);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> checks = new ArrayList<>();
+
+        for (int i = 0; i < criteria1.size(); i++) {
+            for (int j = 0; j < criteria2.size(); j++) {
+                checks.add(new T2<>(
+                    (RangeIndexQueryCriterion)criteria1.get(i),
+                    (RangeIndexQueryCriterion)criteria2.get(j)));
+            }
+
+        }
+
+        checks.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMergeMultipleCriteriaForSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(gt(fldName, lower), gt(fldName, lower - 1), gt(fldName, lower - 2),
+                lt(fldName, upper), lt(fldName, upper + 1), lt(fldName, upper + 2));
+
+        check(null, cache.query(qry), lower + 1, upper);
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        checkEqualsCriteria(lower, upper, 0, CNT);
+        checkEqualsCriteria(lower, upper, CNT, 0);
+        checkEqualsCriteria(upper, lower, 0, CNT);
+        checkEqualsCriteria(upper, lower, CNT, 0);
+    }
+
+    /** */
+    private void checkEqualsCriteria(int eq1, int eq2, int from, int to) {
+        GridTestUtils.assertThrows(null, () -> {
+            IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+                .setCriteria(eq(fldName, eq1), eq(fldName, eq2), between(fldName, from, to));

Review comment:
       see IndexFailover test you mentioned in previous comment, it checks `eq-and-eq` criteria.




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r722088146



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {

Review comment:
       the magic number MUST be linked to the `switch` somehow.




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r716812646



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       Looks like this method partially duplicates `H2RowComparator#compareKey`
   Should it use this method instead?




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {
+                    RangeIndexQueryCriterion c1 = (RangeIndexQueryCriterion)criterion(i, fldName, pair.get1(), pair.get2());
+                    RangeIndexQueryCriterion c2 = (RangeIndexQueryCriterion)criterion(j, fldName, pair.get2(), pair.get1());
+
+                    cc.add(new T2<>(c1, c2));
+                }
+            }
+        });
+
+        cc.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        checkEqualsCriteria(lower, upper, 0, CNT);
+        checkEqualsCriteria(lower, upper, CNT, 0);
+        checkEqualsCriteria(upper, lower, 0, CNT);
+        checkEqualsCriteria(upper, lower, CNT, 0);
+    }
+
+    /** */
+    private void checkEqualsCriteria(int eq1, int eq2, int from, int to) {
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, eq1), eq(fldName, eq2), between(fldName, from, to));
+
+        assertTrue(qry.getCriteria().toString(), cache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testCommonBoundary() {

Review comment:
       Renamed `lower` and `upper` to `boundary`.




-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       > I see we duplicate the following and wish this can be fixed
   
   It's duplicated because we put all logic of comparison related to H2 to single place. Actually there is no much of duplicated code 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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {
+                    RangeIndexQueryCriterion c1 = (RangeIndexQueryCriterion)criterion(i, fldName, pair.get1(), pair.get2());
+                    RangeIndexQueryCriterion c2 = (RangeIndexQueryCriterion)criterion(j, fldName, pair.get2(), pair.get1());
+
+                    cc.add(new T2<>(c1, c2));
+                }
+            }
+        });
+
+        cc.forEach(c -> checkTwoCriteria(c.get1(), c.get2()));
+    }
+
+    /** */
+    @Test
+    public void testMultipleEqualsCriteria() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        checkEqualsCriteria(lower, upper, 0, CNT);
+        checkEqualsCriteria(lower, upper, CNT, 0);
+        checkEqualsCriteria(upper, lower, 0, CNT);
+        checkEqualsCriteria(upper, lower, CNT, 0);
+    }
+
+    /** */
+    private void checkEqualsCriteria(int eq1, int eq2, int from, int to) {
+        IndexQuery<Integer, Person> qry = new IndexQuery<Integer, Person>(Person.class, idxName)
+            .setCriteria(eq(fldName, eq1), eq(fldName, eq2), between(fldName, from, to));
+
+        assertTrue(qry.getCriteria().toString(), cache.query(qry).getAll().isEmpty());
+    }
+
+    /** */
+    @Test
+    public void testCommonBoundary() {

Review comment:
       Why? upper == lower in this test. 




-- 
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] anton-vinogradov commented on a change in pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9429:
URL: https://github.com/apache/ignite/pull/9429#discussion_r722070347



##########
File path: modules/indexing/src/test/java/org/apache/ignite/cache/query/RepeatedFieldIndexQueryTest.java
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+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.internal.cache.query.RangeIndexQueryCriterion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.between;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.eq;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.gte;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lt;
+import static org.apache.ignite.cache.query.IndexQueryCriteriaBuilder.lte;
+
+/** */
+@RunWith(Parameterized.class)
+public class RepeatedFieldIndexQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE = "TEST_CACHE";
+
+    /** */
+    private static final String ID_IDX = "ID_IDX";
+
+    /** */
+    private static final String DESC_ID_IDX = "DESC_ID_IDX";
+
+    /** */
+    private static final int CNT = 10_000;
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** */
+    @Parameterized.Parameter
+    public String idxName;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public String fldName;
+
+    /** */
+    @Parameterized.Parameters(name = "idx={0} fldName={1}")
+    public static List<Object[]> params() {
+        return F.asList(
+            new Object[] {ID_IDX, "id"},
+            new Object[] {DESC_ID_IDX, "descId"}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite crd = startGrids(2);
+
+        cache = crd.cache(CACHE);
+
+        for (int i = 0; i < CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg1 = new CacheConfiguration<>()
+            .setName(CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg1);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testRangeQueriesWithTwoCriteriaSingleField() {
+        int lower = new Random().nextInt(CNT / 2);
+        int upper = CNT / 2 + new Random().nextInt(CNT / 2 - 1);
+
+        List<T2<RangeIndexQueryCriterion, RangeIndexQueryCriterion>> cc = new ArrayList<>();
+
+        Stream.of(new T2<>(lower, upper), new T2<>(upper, lower)).forEach(pair -> {
+            for (int i = 0; i < 6; i++) {
+                for (int j = 0; j < 6; j++) {

Review comment:
       Why not just call `compareKey` from `compareRow` when both.indexSearchRow() (BTW, is it possible that onlyOne.indexSearchRow()?)
   
   I see we duplicate the fillowing 
   ```
           int ltype = DataType.getTypeFromClass(left.key().getClass());
           int rtype = DataType.getTypeFromClass(right.key().getClass());
   
           int c = compareValues(wrap(left.key(), ltype), wrap(right.key(), rtype));
   
           return Integer.signum(c);
   ```
   and wish this can be fixed )




-- 
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] anton-vinogradov merged pull request #9429: IGNITE-15535 Allow specify multiple criteria for the same field

Posted by GitBox <gi...@apache.org>.
anton-vinogradov merged pull request #9429:
URL: https://github.com/apache/ignite/pull/9429


   


-- 
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 #9429: IGNITE-15535 Allow specify multiple criteria for the same field

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
##########
@@ -43,4 +43,12 @@
      * @param idx offset of index key.
      */
     public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index key.
+     * @param right index key.
+     */
+    public int compareSearchKey(IndexKey left, IndexKey right) throws IgniteCheckedException;

Review comment:
       > BTW, is it possible that onlyOne.indexSearchRow()
   
   Yes, it's case of finding range in index rows by search row. In this moment we compare user's search row with stored (non-search) index row.




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