You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by al...@apache.org on 2022/09/30 12:49:21 UTC

[ignite] branch master updated: IGNITE-17791 SQL: Allow index scans for compatible types - Fixes #10281.

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

alexpl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new a26211c7a90 IGNITE-17791 SQL: Allow index scans for compatible types - Fixes #10281.
a26211c7a90 is described below

commit a26211c7a90ca0664cbb0b19b64371c56fe7f42f
Author: Aleksey Plekhanov <pl...@gmail.com>
AuthorDate: Fri Sep 30 15:36:47 2022 +0300

    IGNITE-17791 SQL: Allow index scans for compatible types - Fixes #10281.
    
    Signed-off-by: Aleksey Plekhanov <pl...@gmail.com>
---
 .../processors/query/h2/database/H2TreeIndex.java  | 21 ++++-
 .../cache/index/IndexColumnTypeMismatchTest.java   | 90 +++++++++++++++++-----
 2 files changed, 88 insertions(+), 23 deletions(-)

diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 0e7274b3267..6e2c4945b18 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -39,6 +39,8 @@ import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler
 import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexQueryContext;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyTypeRegistry;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
@@ -282,14 +284,25 @@ public class H2TreeIndex extends H2TreeIndexBase {
 
             // If it's possible to convert search row to index value type - do it. In this case converted value
             // can be used for the inline search.
-            // Otherwise, we can't use search row as index find bound, since different types have different comparison
-            // rules (for example, '2' > '10' for strings and 2 < 10 for integers). Best we can do here is leave search
-            // bound empty. In this case index scan by bounds can be extended to full index scan and rows will be
-            // filtered out by original condition on H2 level.
+            // Otherwise, we can use search row as index find bound only if types have the same comparison rules.
+            // If types have different comparison rules (for example, '2' > '10' for strings and 2 < 10 for integers)
+            // best we can do here is leave search bound empty. In this case index scan by bounds can be extended to
+            // full index scan and rows will be filtered out by original condition on H2 level.
             if (colType.code() != v.getType()) {
                 if (Value.getHigherOrder(colType.code(), v.getType()) == colType.code())
                     v = v.convertTo(colType.code());
                 else {
+                    InlineIndexKeyType colKeyType = InlineIndexKeyTypeRegistry.get(colType, queryIndex.keyTypeSettings());
+
+                    IndexKey idxKey = IndexKeyFactory.wrap(v.getObject(), v.getType(), cctx.cacheObjectContext(),
+                        queryIndex.keyTypeSettings());
+
+                    if (colKeyType.isComparableTo(idxKey)) {
+                        keys[i] = idxKey;
+
+                        continue;
+                    }
+
                     LT.warn(log, "Provided value can't be used as index search bound due to column data type " +
                         "mismatch. This can lead to full index scans instead of range index scans. [index=" +
                         idxName + ", colType=" + colType + ", valType=" + IndexKeyType.forCode(v.getType()) + ']');
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IndexColumnTypeMismatchTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IndexColumnTypeMismatchTest.java
index da29ad2c8dd..17eef45ce58 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IndexColumnTypeMismatchTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IndexColumnTypeMismatchTest.java
@@ -17,7 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache.index;
 
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Calendar;
 import java.util.List;
+import java.util.function.IntFunction;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
@@ -29,6 +33,10 @@ import org.junit.Test;
  * Test query on index by search row with mismatched column type.
  */
 public class IndexColumnTypeMismatchTest extends AbstractIndexingCommonTest {
+    /** */
+    private static final String LOG_MSG =
+        "Provided value can't be used as index search bound due to column data type mismatch";
+
     /** */
     private final ListeningTestLogger listeningLog = new ListeningTestLogger(log);
 
@@ -40,42 +48,86 @@ public class IndexColumnTypeMismatchTest extends AbstractIndexingCommonTest {
         return super.getConfiguration(igniteInstanceName).setGridLogger(listeningLog);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
     /** */
     @Test
     public void testIndexColTypeMismatch() throws Exception {
-        LogListener lsnr = LogListener
-            .matches("Provided value can't be used as index search bound due to column data type mismatch")
-            .times(1)
-            .build();
+        LogListener lsnr = LogListener.matches(LOG_MSG).times(1).build();
 
         listeningLog.registerListener(lsnr);
 
         IgniteEx ignite = startGrid(0);
-        String cacheName = "test";
-
-        sql(ignite, "CREATE TABLE test (id INTEGER, val VARCHAR, PRIMARY KEY (id)) " +
-            "WITH \"CACHE_NAME=" + cacheName + "\"");
 
-        sql(ignite, "CREATE INDEX test_idx ON test (val)");
+        checkIndexQuery(ignite, "VARCHAR", String::valueOf, i -> i);
 
-        for (int i = 0; i < ROW_COUNT; i++)
-            sql(ignite, "INSERT INTO test VALUES (?, ?)", i, i);
+        assertTrue(lsnr.check());
+    }
 
-        for (int i = 0; i < ROW_COUNT; i++) {
-            // Use 'int' as a search row for 'string' index.
-            List<List<?>> res = sql(ignite, "SELECT val FROM test WHERE val = ?", i);
+    /** */
+    @Test
+    public void testIndexDifferentTypesComparable() throws Exception {
+        LogListener lsnr = LogListener.matches(LOG_MSG).times(0).build();
 
-            assertEquals(1, res.size());
-            assertEquals(String.valueOf(i), res.get(0).get(0));
-        }
+        listeningLog.registerListener(lsnr);
 
-        List<List<?>> res = sql(ignite, "SELECT val FROM test WHERE val < ?", 50);
+        IgniteEx ignite = startGrid(0);
 
-        assertEquals(50, res.size());
+        checkIndexQuery(ignite, "INT", i -> i, i -> (long)i);
+        checkIndexQuery(ignite, "BIGINT", i -> (long)i, i -> i);
+        checkIndexQuery(ignite, "DATE", i -> new Date(millis(i)), i -> new Timestamp(millis(i)));
+        checkIndexQuery(ignite, "TIMESTAMP", i -> new Timestamp(millis(i)), i -> new Date(millis(i)));
+        checkIndexQuery(ignite, "INT", i -> i, String::valueOf);
 
         assertTrue(lsnr.check());
     }
 
+    /** */
+    private void checkIndexQuery(
+        IgniteEx ignite,
+        String indexedType,
+        IntFunction<Object> indexedValFactory,
+        IntFunction<Object> searchRowFactory
+    ) {
+        try {
+            sql(ignite, "CREATE TABLE test (id INTEGER, val " + indexedType +
+                ", PRIMARY KEY (id)) WITH \"CACHE_NAME=test\"");
+
+            sql(ignite, "CREATE INDEX test_idx ON test (val)");
+
+            for (int i = 0; i < ROW_COUNT; i++)
+                sql(ignite, "INSERT INTO test VALUES (?, ?)", i, indexedValFactory.apply(i));
+
+            for (int i = 0; i < ROW_COUNT; i++) {
+                List<List<?>> res = sql(ignite, "SELECT val FROM test WHERE val = ?", searchRowFactory.apply(i));
+
+                assertEquals(1, res.size());
+                assertEquals(indexedValFactory.apply(i), res.get(0).get(0));
+            }
+
+            List<List<?>> res = sql(ignite, "SELECT val FROM test WHERE val < ?", searchRowFactory.apply(50));
+
+            assertEquals(50, res.size());
+        }
+        finally {
+            sql(ignite, "DROP TABLE test");
+        }
+    }
+
+    /** */
+    private long millis(int day) {
+        Calendar cal = Calendar.getInstance();
+        cal.clear();
+        cal.set(2022, Calendar.JANUARY, 1);
+        cal.add(Calendar.DATE, day);
+        return cal.getTimeInMillis();
+    }
+
     /** */
     private List<List<?>> sql(IgniteEx ign, String sql, Object... args) {
         return ign.context().query().querySqlFields(new SqlFieldsQuery(sql).setArgs(args), false).getAll();