You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tl...@apache.org on 2020/08/26 08:20:15 UTC

[ignite] branch master updated: IGNITE-13376 fix fields sequence at the PK (#8172)

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

tledkov 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 301b5cf  IGNITE-13376 fix fields sequence at the PK (#8172)
301b5cf is described below

commit 301b5cf948c3f2c583404181bf469856ef31b679
Author: Stanilovsky Evgeny <st...@gmail.com>
AuthorDate: Wed Aug 26 11:19:52 2020 +0300

    IGNITE-13376 fix fields sequence at the PK (#8172)
---
 .../java/org/apache/ignite/cache/QueryEntity.java  |  3 +-
 .../org/apache/ignite/internal/IgniteFeatures.java |  5 +-
 .../processors/query/GridQueryTypeDescriptor.java  | 13 ++++
 .../processors/query/QueryTypeDescriptorImpl.java  | 14 ++++
 .../internal/processors/query/QueryUtils.java      | 17 ++++-
 .../processors/query/h2/H2TableDescriptor.java     | 23 +++++-
 .../processors/cache/index/BasicIndexTest.java     | 86 ++++++++++++++++++++++
 .../processors/query/SqlSystemViewsSelfTest.java   |  2 +-
 8 files changed, 153 insertions(+), 10 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index affff27..7f71f1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -122,7 +123,7 @@ public class QueryEntity implements Serializable {
         valueFieldName = other.valueFieldName;
 
         fields = new LinkedHashMap<>(other.fields);
-        keyFields = other.keyFields != null ? new HashSet<>(other.keyFields) : null;
+        keyFields = other.keyFields != null ? new LinkedHashSet<>(other.keyFields) : null;
 
         aliases = new HashMap<>(other.aliases);
         idxs = other.idxs != null ? new ArrayList<>(other.idxs) : null;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
index 4196435..c99f528 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
@@ -121,7 +121,10 @@ public enum IgniteFeatures {
     INDEXES_MANIPULATIONS_FROM_CONTROL_SCRIPT(42),
 
     /** Optimization of recovery protocol for cluster which doesn't contain MVCC caches. */
-    MVCC_TX_RECOVERY_PROTOCOL_V2(44);
+    MVCC_TX_RECOVERY_PROTOCOL_V2(44),
+
+    /** Pk index keys are applied in correct order. */
+    SPECIFIED_SEQ_PK_KEYS(45);
 
     /**
      * Unique feature identifier.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index 6a40359..c8b2cc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query;
 
 import java.util.Map;
+import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.jetbrains.annotations.Nullable;
@@ -198,4 +199,16 @@ public interface GridQueryTypeDescriptor {
      * @throws IgniteCheckedException If failed.
      */
     public void setDefaults(Object key, Object val) throws IgniteCheckedException;
+
+    /**
+     * Gets primary key fields if defined, or empty collection otherwise.
+     */
+    public Set<String> primaryKeyFields();
+
+    /**
+     * Sets primary key fields.
+     *
+     * @param keys Primary keys.
+     */
+    public void primaryKeyFields(Set<String> keys);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index 21e0154..f53b38c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.QueryIndexType;
@@ -131,6 +132,9 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     /** */
     private final CacheObjectContext coCtx;
 
+    /** Primary key fields. */
+    private Set<String> pkFields;
+
     /**
      * Constructor.
      *
@@ -668,4 +672,14 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
             prop.setValue(key, val, prop.defaultValue());
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Set<String> primaryKeyFields() {
+        return pkFields == null ? Collections.emptySet() : pkFields;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void primaryKeyFields(Set<String> keys) {
+        pkFields = keys;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index efd9183..c5dba5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -625,6 +625,9 @@ public class QueryUtils {
             d.addProperty(prop, false);
         }
 
+        if (!isKeyClsSqlType)
+            d.primaryKeyFields(keyFields);
+
         // Sql-typed key/value doesn't have field property, but they may have precision and scale constraints.
         // Also if fields are not set then _KEY and _VAL will be created as visible,
         // so we have to add binary properties for them
@@ -811,9 +814,17 @@ public class QueryUtils {
      * @param scale Scale.
      * @return Binary property.
      */
-    public static QueryBinaryProperty buildBinaryProperty(GridKernalContext ctx, String pathStr,
-        Class<?> resType, Map<String, String> aliases, boolean isKeyField, boolean notNull, Object dlftVal,
-        int precision, int scale) {
+    public static QueryBinaryProperty buildBinaryProperty(
+        GridKernalContext ctx,
+        String pathStr,
+        Class<?> resType,
+        Map<String, String> aliases,
+        boolean isKeyField,
+        boolean notNull,
+        Object dlftVal,
+        int precision,
+        int scale
+    ) {
         String[] path = pathStr.split("\\.");
 
         QueryBinaryProperty res = null;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
index 2720a44..4f7117a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
@@ -327,15 +327,30 @@ public class H2TableDescriptor {
             if (QueryUtils.isSqlType(type.keyClass()))
                 keyCols.add(keyCol);
             else {
-                for (String propName : type.fields().keySet()) {
-                    GridQueryProperty prop = type.property(propName);
+                // SPECIFIED_SEQ_PK_KEYS check guarantee that request running on heterogeneous (RU) cluster can
+                // perform equally on all nodes.
+                if (!idx.kernalContext().recoveryMode()) {
+                    for (String keyName : type.primaryKeyFields()) {
+                        GridQueryProperty prop = type.property(keyName);
 
-                    if (prop.key()) {
-                        Column col = tbl.getColumn(propName);
+                        assert prop.key() : keyName + " is not a key field";
+
+                        Column col = tbl.getColumn(prop.name());
 
                         keyCols.add(tbl.indexColumn(col.getColumnId(), SortOrder.ASCENDING));
                     }
                 }
+                else {
+                    for (String propName : type.fields().keySet()) {
+                        GridQueryProperty prop = type.property(propName);
+
+                        if (prop.key()) {
+                            Column col = tbl.getColumn(propName);
+
+                            keyCols.add(tbl.indexColumn(col.getColumnId(), SortOrder.ASCENDING));
+                        }
+                    }
+                }
 
                 // If key is object but the user has not specified any particular columns,
                 // we have to fall back to whole-key index.
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java
index ee28b71..8725e3a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java
@@ -26,6 +26,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Objects;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -43,12 +44,15 @@ import org.apache.ignite.failure.StopNodeFailureHandler;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.ListeningTestLogger;
 import org.apache.ignite.testframework.LogListener;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.h2.index.Index;
+import org.h2.table.Column;
 import org.jetbrains.annotations.Nullable;
 import org.junit.Test;
 
@@ -269,6 +273,65 @@ public class BasicIndexTest extends AbstractIndexingCommonTest {
     }
 
     /**
+     * Checks that fields in primary index have correct order.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCorrectPrimaryKeyFieldsSequence() throws Exception {
+        inlineSize = 10;
+
+        IgniteEx ig0 = startGrid(0);
+
+        GridQueryProcessor qryProc = ig0.context().query();
+
+        IgniteH2Indexing idx = (IgniteH2Indexing)(ig0).context().query().getIndexing();
+
+        String tblName = "T1";
+
+        qryProc.querySqlFields(new SqlFieldsQuery("CREATE TABLE PUBLIC." + tblName + " (F1 VARCHAR, F2 VARCHAR, F3 VARCHAR, " +
+            "CONSTRAINT PK PRIMARY KEY (F1, F2))"), true).getAll();
+
+        List<String> expect = Arrays.asList("F1", "F2");
+
+        checkPkFldSequence(tblName, expect, idx);
+
+        tblName = "T2";
+
+        qryProc.querySqlFields(new SqlFieldsQuery("CREATE TABLE PUBLIC." + tblName + " (F1 VARCHAR, F2 VARCHAR, F3 VARCHAR, " +
+            "CONSTRAINT PK PRIMARY KEY (F2, F1))"), true).getAll();
+
+        expect = Arrays.asList("F2", "F1");
+
+        checkPkFldSequence(tblName, expect, idx);
+
+        tblName = "T3";
+
+        qryProc.querySqlFields(new SqlFieldsQuery("CREATE TABLE PUBLIC." + tblName + " (F1 VARCHAR, F2 VARCHAR, F3 VARCHAR, " +
+            "CONSTRAINT PK PRIMARY KEY (F3, F2))"), true).getAll();
+
+        expect = Arrays.asList("F3", "F2");
+
+        checkPkFldSequence(tblName, expect, idx);
+    }
+
+    /**
+     * Fields correctness checker.
+     *
+     * @param tblName Table name.
+     * @param expect Expected fields sequence.
+     * @param idx Indexing.
+     */
+    private void checkPkFldSequence(String tblName, List<String> expect, IgniteH2Indexing idx) {
+        Index pkIdx = idx.schemaManager().dataTable("PUBLIC", tblName.toUpperCase()).getIndex(PK_IDX_NAME);
+
+        List<String> actual = Arrays.stream(pkIdx.getColumns()).map(Column::getName).collect(Collectors.toList());
+
+        if (!expect.equals(actual))
+            throw new AssertionError("Exp: " + expect + ", but was: " + actual);
+    }
+
+    /**
      * Tests mixed dynamic and static caches with indexes creation.
      *
      * @throws Exception If failed.
@@ -535,6 +598,29 @@ public class BasicIndexTest extends AbstractIndexingCommonTest {
     }
 
     /**
+     *  Checks index usage with correct pk fields enumeration.
+     */
+    @Test
+    public void testCorrectFieldsSequenceInPk() throws Exception {
+        inlineSize = 10;
+
+        srvLog = new ListeningTestLogger(false, log);
+
+        IgniteEx ig0 = startGrid(0);
+
+        GridQueryProcessor qryProc = ig0.context().query();
+
+        populateTable(qryProc, TEST_TBL_NAME, -2, "FIRST_NAME", "LAST_NAME",
+            "ADDRESS", "LANG");
+
+        assertFalse(checkIdxAlreadyExistLog(
+            qryProc, "idx1", TEST_TBL_NAME, "FIRST_NAME", "LAST_NAME"));
+
+        assertTrue(checkIdxAlreadyExistLog(
+            qryProc, "idx2", TEST_TBL_NAME, "LAST_NAME", "FIRST_NAME"));
+    }
+
+    /**
      *  Checks index usage for full coverage.
      */
     @Test
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
index 92034ea..c4e0b12 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
@@ -322,7 +322,7 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
             assertEquals(expRow.length, resRow.size());
 
             for (int j = 0; j < expRow.length; j++)
-                assertEquals(expRow[j], String.valueOf(resRow.get(j)));
+                assertEquals(Integer.toString(i), expRow[j], String.valueOf(resRow.get(j)));
         }
     }