You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/02/19 14:23:52 UTC

ignite git commit: ignite-1232 Index for affinity key field should be non-unique

Repository: ignite
Updated Branches:
  refs/heads/ignite-1232 7fab7051a -> 6c91a2a4e


ignite-1232 Index for affinity key field should be non-unique


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

Branch: refs/heads/ignite-1232
Commit: 6c91a2a4e2c3540a62d17e5c1364c8cad0db3183
Parents: 7fab705
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 19 15:17:59 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 19 15:17:59 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +-
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../cache/IgniteCacheJoinQueryTest.java         | 176 +++++++++++++++----
 3 files changed, 146 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6c91a2a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 1215141..d2e5df2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -2337,7 +2337,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             // Add explicit affinity key index if nothing alike was found.
             if (affCol != null && !affIdxFound) {
-                idxs.add(new GridH2TreeIndex("AFFINITY_KEY", tbl, true,
+                idxs.add(new GridH2TreeIndex("AFFINITY_KEY", tbl, false,
                     treeIndexColumns(new ArrayList<IndexColumn>(2), affCol, keyCol)));
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c91a2a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index b4a056c..3592928 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -573,7 +573,7 @@ public class GridH2Table extends TableBase {
                 while (++i < len) {
                     GridH2IndexBase idx = index(i);
 
-                    assert !idx.getIndexType().isUnique() : "Unique indexes are not supported.";
+                    assert !idx.getIndexType().isUnique() : "Unique indexes are not supported: " + idx;
 
                     GridH2Row old2 = idx.put(row);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c91a2a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryTest.java
index 6daa2e1..d9fee72 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryTest.java
@@ -26,14 +26,17 @@ import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 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.CacheMode;
 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.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -67,7 +70,7 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
 
         CacheKeyConfiguration keyCfg = new CacheKeyConfiguration();
 
-        keyCfg.setTypeName(PersonKeyWithAffinity.class.getName());
+        keyCfg.setTypeName(TestKeyWithAffinity.class.getName());
         keyCfg.setAffinityKeyFieldName("affKey");
 
         cfg.setCacheKeyConfiguration(keyCfg);
@@ -167,13 +170,22 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
     public void testJoinQuery(CacheMode cacheMode, int backups, boolean affKey) {
         CacheConfiguration ccfg = cacheConfiguration(cacheMode, backups, affKey, affKey);
 
+        log.info("Test cache [mode=" + cacheMode + ", backups=" + backups + ']');
+
         IgniteCache cache = ignite(0).createCache(ccfg);
 
         try {
-            Map<Integer, Integer> cnts = putData(cache, affKey);
+            PutData putData = putData(cache, affKey);
+
+            for (int i = 0; i < NODES; i++) {
+                log.info("Test node: " + i);
+
+                IgniteCache cache0 = ignite(i).cache(ccfg.getName());
+
+                checkPersonAccountsJoin(cache0, putData.personAccounts, affKey);
 
-            for (int i = 0; i < NODES; i++)
-                checkJoin(ignite(i).cache(ccfg.getName()), cnts);
+                checkOrganizationPersonsJoin(cache, putData.orgPersons);
+            }
         }
         finally {
             ignite(0).destroyCache(ccfg.getName());
@@ -182,21 +194,48 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Cache.
-     * @param personCnt Count of persons per organization.
+     * @param cnts Organizations per person counts.
      */
-    private void checkJoin(IgniteCache cache, Map<Integer, Integer> personCnt) {
+    private void checkOrganizationPersonsJoin(IgniteCache cache, Map<Integer, Integer> cnts) {
         SqlFieldsQuery qry = new SqlFieldsQuery("select o.name, p.name " +
             "from Organization o, Person p " +
             "where p.orgId = o._key and o._key=?");
 
         qry.setDistributedJoins(true);
 
-        for (int i = 0; i < personCnt.size(); i++) {
+        for (int i = 0; i < cnts.size(); i++) {
             qry.setArgs(i);
 
             List<List<Object>> res = cache.query(qry).getAll();
 
-            assertEquals((int)personCnt.get(i), res.size());
+            assertEquals((int)cnts.get(i), res.size());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param cnts Accounts per person counts.
+     * @param affKey If {@code true} uses key with affinity key field.
+     */
+    private void checkPersonAccountsJoin(IgniteCache cache, Map<Object, Integer> cnts, boolean affKey) {
+        SqlFieldsQuery qry = new SqlFieldsQuery("select p.name " +
+            "from Person p, " + (affKey ? "AccountKeyWithAffinity" : "Account") + " a " +
+            "where p._key = a.personId and p._key=?");
+
+        qry.setDistributedJoins(true);
+
+        Ignite ignite = (Ignite)cache.unwrap(Ignite.class);
+
+        boolean binary = ignite.configuration().getMarshaller() instanceof BinaryMarshaller;
+
+        for (Map.Entry<Object, Integer> e : cnts.entrySet()) {
+            Object arg = binary ? ignite.binary().toBinary(e.getKey()) : e.getKey();
+
+            qry.setArgs(arg);
+
+            List<List<Object>> res = cache.query(qry).getAll();
+
+            assertEquals((int)e.getValue(), res.size());
         }
     }
 
@@ -220,8 +259,15 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
 
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
+        String personKeyType = affKey ? TestKeyWithAffinity.class.getName() : TestKey.class.getName();
+
+        QueryEntity account = new QueryEntity();
+        account.setKeyType(Integer.class.getName());
+        account.setValueType(affKey ? AccountKeyWithAffinity.class.getName() : Account.class.getName());
+        account.addQueryField("personId", personKeyType, null);
+
         QueryEntity person = new QueryEntity();
-        person.setKeyType(affKey ? PersonKeyWithAffinity.class.getName() : PersonKey.class.getName());
+        person.setKeyType(personKeyType);
         person.setValueType(Person.class.getName());
         person.addQueryField("orgId", Integer.class.getName(), null);
         person.addQueryField("id", Integer.class.getName(), null);
@@ -235,7 +281,7 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
         org.setValueType(Organization.class.getName());
         org.addQueryField("name", String.class.getName(), null);
 
-        ccfg.setQueryEntities(F.asList(person, org));
+        ccfg.setQueryEntities(F.asList(account, person, org));
 
         return ccfg;
     }
@@ -243,59 +289,87 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
     /**
      * @param cache Cache.
      * @param affKey If {@code true} uses key with affinity key field.
-     * @return Count of persons per organization.
+     * @return Put data counts.
      */
-    private Map<Integer, Integer> putData(IgniteCache cache, boolean affKey) {
-        Map<Integer, Integer> personCnt = new HashMap<>();
+    private PutData putData(IgniteCache cache, boolean affKey) {
+        Map<Integer, Integer> orgPersons = new HashMap<>();
+        Map<Object, Integer> personAccounts = new HashMap<>();
 
         final int ORG_CNT = 10;
 
         for (int i = 0; i < ORG_CNT; i++)
             cache.put(i, new Organization("org-" + i));
 
-        Set<Integer> ids = new HashSet<>();
+        Set<Integer> personIds = new HashSet<>();
+        Set<Integer> accountIds = new HashSet<>();
 
         for (int i = 0; i < ORG_CNT; i++) {
-            int cnt = ThreadLocalRandom.current().nextInt(100);
+            int persons = ThreadLocalRandom.current().nextInt(100);
 
-            for (int j = 0; j < cnt; j++) {
+            for (int p = 0; p < persons; p++) {
                 int personId = ThreadLocalRandom.current().nextInt();
 
-                while (!ids.add(personId))
+                while (!personIds.add(personId))
                     personId = ThreadLocalRandom.current().nextInt();
 
-                Object key = affKey ? new PersonKeyWithAffinity(personId) : new PersonKey(personId);
+                Object personKey = affKey ? new TestKeyWithAffinity(personId) : new TestKey(personId);
 
                 String name = "person-" + personId;
 
-                cache.put(key, new Person(i, name));
+                cache.put(personKey, new Person(i, name));
+
+                int accounts = ThreadLocalRandom.current().nextInt(10);
+
+                for (int a = 0; a < accounts; a++) {
+                    int accountId = ThreadLocalRandom.current().nextInt();
+
+                    while (!accountIds.add(accountId))
+                        accountId = ThreadLocalRandom.current().nextInt();
 
-                assertEquals(name, ((Person)cache.get(key)).name);
+                    cache.put(accountId, affKey ? new AccountKeyWithAffinity(personKey) : new Account(personKey));
+                }
+
+                personAccounts.put(personKey, accounts);
             }
 
-            personCnt.put(i, cnt);
+            orgPersons.put(i, persons);
         }
 
-        return personCnt;
+        return new PutData(orgPersons, personAccounts);
     }
 
     /**
      *
      */
-    public static class PersonKeyWithAffinity {
+    private static class PutData {
         /** */
-        private int id;
+        final Map<Integer, Integer> orgPersons;
 
         /** */
-        private int affKey;
+        final Map<Object, Integer> personAccounts;
+
+        /**
+         * @param orgPersons Organizations per person counts.
+         * @param personAccounts Accounts per person counts.
+         */
+        public PutData(Map<Integer, Integer> orgPersons, Map<Object, Integer> personAccounts) {
+            this.orgPersons = orgPersons;
+            this.personAccounts = personAccounts;
+        }
+    }
+
+    /**
+     *
+     */
+    public static class TestKey {
+        /** */
+        private int id;
 
         /**
          * @param id Key.
          */
-        public PersonKeyWithAffinity(int id) {
+        public TestKey(int id) {
             this.id = id;
-
-            affKey = id + 1;
         }
 
         /** {@inheritDoc} */
@@ -306,7 +380,7 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
             if (o == null || getClass() != o.getClass())
                 return false;
 
-            PersonKeyWithAffinity other = (PersonKeyWithAffinity)o;
+            TestKey other = (TestKey)o;
 
             return id == other.id;
         }
@@ -320,15 +394,21 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
     /**
      *
      */
-    public static class PersonKey {
+    public static class TestKeyWithAffinity {
         /** */
         private int id;
 
+        /** */
+        @AffinityKeyMapped
+        private int affKey;
+
         /**
          * @param id Key.
          */
-        public PersonKey(int id) {
+        public TestKeyWithAffinity(int id) {
             this.id = id;
+
+            affKey = id + 1;
         }
 
         /** {@inheritDoc} */
@@ -339,7 +419,7 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
             if (o == null || getClass() != o.getClass())
                 return false;
 
-            PersonKey other = (PersonKey)o;
+            TestKeyWithAffinity other = (TestKeyWithAffinity)o;
 
             return id == other.id;
         }
@@ -353,6 +433,38 @@ public class IgniteCacheJoinQueryTest extends GridCommonAbstractTest {
     /**
      *
      */
+    private static class Account implements Serializable {
+        /** */
+        @QuerySqlField
+        private TestKey personId;
+
+        /**
+         * @param personId Person ID.
+         */
+        public Account(Object personId) {
+            this.personId = (TestKey)personId;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class AccountKeyWithAffinity implements Serializable {
+        /** */
+        @QuerySqlField
+        private TestKeyWithAffinity personId;
+
+        /**
+         * @param personId Person ID.
+         */
+        public AccountKeyWithAffinity(Object personId) {
+            this.personId = (TestKeyWithAffinity)personId;
+        }
+    }
+
+    /**
+     *
+     */
     private static class Person implements Serializable {
         /** */
         @QuerySqlField