You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2018/11/06 14:09:36 UTC

ignite git commit: IGNITE-7793: Made SQL work what a value object has a field which name equals to affinity key name. This closes #4770.

Repository: ignite
Updated Branches:
  refs/heads/master 362d51525 -> dcdb27a24


IGNITE-7793: Made SQL work what a value object has a field which name equals to affinity key name. This closes #4770.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/dcdb27a2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/dcdb27a2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/dcdb27a2

Branch: refs/heads/master
Commit: dcdb27a24a450f63487290360b265e1570534629
Parents: 362d515
Author: Max-Pudov <pu...@gmail.com>
Authored: Tue Nov 6 17:09:27 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Nov 6 17:09:27 2018 +0300

----------------------------------------------------------------------
 .../internal/processors/query/QueryUtils.java   |  15 +-
 .../query/property/QueryBinaryProperty.java     |   8 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  25 ++
 ...ityKeyNameAndValueFieldNameConflictTest.java | 261 +++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java        |   3 +
 5 files changed, 297 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dcdb27a2/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
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 44e07a6..e530ab1 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
@@ -827,17 +827,10 @@ public class QueryUtils {
     public static QueryClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr,
         Class<?> resType, Map<String,String> aliases, boolean notNull, CacheObjectContext coCtx)
         throws IgniteCheckedException {
-        QueryClassProperty res = buildClassProperty(
-            true,
-            keyCls,
-            pathStr,
-            resType,
-            aliases,
-            notNull,
-            coCtx);
-
-        if (res == null) // We check key before value consistently with BinaryProperty.
-            res = buildClassProperty(false, valCls, pathStr, resType, aliases, notNull, coCtx);
+        QueryClassProperty res = buildClassProperty(false, valCls, pathStr, resType, aliases, notNull, coCtx);
+
+        if (res == null) // We check value before key consistently with BinaryProperty.
+            res = buildClassProperty(true, keyCls, pathStr, resType, aliases, notNull, coCtx);
 
         if (res == null)
             throw new IgniteCheckedException(propertyInitializationExceptionMessage(keyCls, valCls, pathStr, resType));

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcdb27a2/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
index 7a47c2f..3f2d233 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
@@ -131,11 +131,11 @@ public class QueryBinaryProperty implements GridQueryProperty {
 
             if (isKeyProp0 == 0) {
                 // Key is allowed to be a non-binary object here.
-                // We check key before value consistently with ClassProperty.
-                if (key instanceof BinaryObject && ((BinaryObject)key).hasField(propName))
-                    isKeyProp = isKeyProp0 = 1;
-                else if (val instanceof BinaryObject && ((BinaryObject)val).hasField(propName))
+                // We check value before key consistently with ClassProperty.
+                if (val instanceof BinaryObject && ((BinaryObject)val).hasField(propName))
                     isKeyProp = isKeyProp0 = -1;
+                else if (key instanceof BinaryObject && ((BinaryObject)key).hasField(propName))
+                    isKeyProp = isKeyProp0 = 1;
                 else {
                     if (!warned) {
                         U.warn(log, "Neither key nor value have property \"" + propName + "\" " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcdb27a2/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 2e27f93..9987a0d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -59,6 +59,7 @@ import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2CollocationModel.isCollocated;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst.TRUE;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST;
@@ -2468,6 +2469,9 @@ public class GridSqlQuerySplitter {
 
         GridH2Table tbl = (GridH2Table)leftCol.getTable();
 
+        if (!isAffinityKey(leftCol.getColumnId(), tbl))
+            return null;
+
         GridH2RowDescriptor desc = tbl.rowDescriptor();
 
         IndexColumn affKeyCol = tbl.getAffinityKeyColumn();
@@ -2502,6 +2506,27 @@ public class GridSqlQuerySplitter {
     }
 
     /**
+     *
+     * @param colId Column ID to check
+     * @param tbl H2 Table
+     * @return is affinity key or not
+     */
+    private static boolean isAffinityKey(int colId, GridH2Table tbl) {
+        GridH2RowDescriptor desc = tbl.rowDescriptor();
+
+        if (desc.isKeyColumn(colId))
+            return true;
+
+        IndexColumn affKeyCol = tbl.getAffinityKeyColumn();
+
+        try {
+            return affKeyCol != null && colId >= DEFAULT_COLUMNS_COUNT && desc.isColumnKeyProperty(colId - DEFAULT_COLUMNS_COUNT) && colId == affKeyCol.column.getColumnId();
+        } catch(IllegalStateException e) {
+            return false;
+        }
+    }
+
+    /**
      * Merges two partition info arrays, removing duplicates
      *
      * @param a Partition info array.

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcdb27a2/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/AffinityKeyNameAndValueFieldNameConflictTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/AffinityKeyNameAndValueFieldNameConflictTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/AffinityKeyNameAndValueFieldNameConflictTest.java
new file mode 100644
index 0000000..67db94c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/AffinityKeyNameAndValueFieldNameConflictTest.java
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+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.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * IGNITE-7793 SQL does not work if value has sql field which name equals to affinity keyProducer name
+ */
+public class AffinityKeyNameAndValueFieldNameConflictTest extends GridCommonAbstractTest {
+    /** */
+    private static final String PERSON_CACHE = "person";
+
+    /** */
+    private Class<?> keyCls;
+
+    /** */
+    private BiFunction<Integer, String, ?> keyProducer;
+
+    /** */
+    private boolean qryEntityCfg;
+
+    /** */
+    private boolean keyFieldSpecified;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration ccfg = new CacheConfiguration(PERSON_CACHE);
+
+        if (qryEntityCfg) {
+            CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(keyCls.getName(), "name");
+            cfg.setCacheKeyConfiguration(keyCfg);
+
+            QueryEntity entity = new QueryEntity();
+            entity.setKeyType(keyCls.getName());
+            entity.setValueType(Person.class.getName());
+            if (keyFieldSpecified)
+                entity.setKeyFields(Stream.of("name").collect(Collectors.toSet()));
+
+            entity.addQueryField("id", Integer.class.getName(), null);
+            entity.addQueryField("name", String.class.getName(), null);
+
+            ccfg.setQueryEntities(F.asList(entity));
+        } else {
+            CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(keyCls);
+            cfg.setCacheKeyConfiguration(keyCfg);
+
+            ccfg.setIndexedTypes(keyCls, Person.class);
+        }
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryEntityConfig() throws Exception {
+        qryEntityCfg = true;
+        keyCls = PersonKey1.class;
+        keyProducer = PersonKey1::new;
+        checkQuery();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryEntityConfigKeySpecified() throws Exception {
+        qryEntityCfg = true;
+        keyFieldSpecified = true;
+        keyCls = PersonKey1.class;
+        keyProducer = PersonKey1::new;
+        checkQuery();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAnnotationConfig() throws Exception {
+        keyCls = PersonKey1.class;
+        keyProducer = PersonKey1::new;
+        checkQuery();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAnnotationConfigCollision() throws Exception {
+        keyCls = PersonKey2.class;
+        keyProducer = PersonKey2::new;
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                checkQuery();
+
+                return null;
+            }
+        }, CacheException.class, "Property with name 'name' already exists.");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkQuery() throws Exception {
+        startGrid(2);
+
+        Ignite g = grid(2);
+
+        IgniteCache<Object, Object> personCache = g.cache(PERSON_CACHE);
+
+        personCache.put(keyProducer.apply(1, "o1"), new Person("p1"));
+
+        SqlFieldsQuery query = new SqlFieldsQuery("select * from \"" + PERSON_CACHE + "\"." + Person.class.getSimpleName() + " it where it.name=?");
+
+        List<List<?>> result = personCache.query(query.setArgs(keyFieldSpecified ? "o1" : "p1")).getAll();
+
+        assertEquals(1, result.size());
+
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    public static class PersonKey1 {
+        /** */
+        @QuerySqlField
+        private int id;
+
+        /** */
+        @AffinityKeyMapped
+        private String name;
+
+        /**
+         * @param id Key.
+         * @param name Affinity keyProducer.
+         */
+        public PersonKey1(int id, String name) {
+            this.id = id;
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            PersonKey1 other = (PersonKey1)o;
+
+            return id == other.id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+    }
+
+    /**
+     *
+     */
+    public static class PersonKey2 {
+        /** */
+        @QuerySqlField
+        private int id;
+
+        /** */
+        @QuerySqlField
+        @AffinityKeyMapped
+        private String name;
+
+        /**
+         * @param id Key.
+         * @param name Affinity keyProducer.
+         */
+        public PersonKey2(int id, String name) {
+            this.id = id;
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            PersonKey2 other = (PersonKey2)o;
+
+            return id == other.id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Person implements Serializable {
+
+        /** */
+        @QuerySqlField
+        String name;
+
+        /**
+         * @param name name.
+         */
+        public Person(String name) {
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Person.class, this);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcdb27a2/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 2fe9323..f1ff5e6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.AffinityKeyNameAndValueFieldNameConflictTest;
 import org.apache.ignite.internal.processors.cache.BigEntryQueryTest;
 import org.apache.ignite.internal.processors.cache.BinaryMetadataConcurrentUpdateWithIndexesTest;
 import org.apache.ignite.internal.processors.cache.BinarySerializationQuerySelfTest;
@@ -238,6 +239,8 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite");
 
+        suite.addTestSuite(AffinityKeyNameAndValueFieldNameConflictTest.class);
+
         suite.addTestSuite(PartitionedSqlTest.class);
         suite.addTestSuite(ReplicatedSqlTest.class);