You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/26 09:31:31 UTC

[10/29] ignite git commit: ignite-2.0 - SQL: Geo tests fixed + distributed join on replicated cache

ignite-2.0 - SQL: Geo tests fixed + distributed join on replicated cache


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

Branch: refs/heads/ignite-5024
Commit: b58d584290fc4ca5f18031fac582130ffab59a24
Parents: de421ff
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Apr 24 16:31:25 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Apr 24 16:31:25 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheProxy.java      |  8 ----
 .../query/h2/H2IndexingAbstractGeoSelfTest.java |  2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  2 +-
 .../query/IgniteSqlSplitterSelfTest.java        | 46 ++++++++++++++++----
 4 files changed, 40 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b58d5842/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 186212d..b38520d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -774,10 +774,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                     opCtxCall != null && opCtxCall.isKeepBinary());
 
             if (qry instanceof SqlQuery) {
-                if (isReplicatedDataNode() && ((SqlQuery)qry).isDistributedJoins())
-                    throw new CacheException("Queries using distributed JOINs have to be run on partitioned cache, " +
-                        "not on replicated.");
-
                 final SqlQuery p = (SqlQuery)qry;
 
                 if ((p.isReplicatedOnly() && isReplicatedDataNode()) || ctx.isLocal() || qry.isLocal())
@@ -788,10 +784,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
             }
 
             if (qry instanceof SqlFieldsQuery) {
-                if (isReplicatedDataNode() && ((SqlFieldsQuery)qry).isDistributedJoins())
-                    throw new CacheException("Queries using distributed JOINs have to be run on partitioned cache, " +
-                        "not on replicated.");
-
                 SqlFieldsQuery p = (SqlFieldsQuery)qry;
 
                 if ((p.isReplicatedOnly() && isReplicatedDataNode()) || ctx.isLocal() || qry.isLocal())

http://git-wip-us.apache.org/repos/asf/ignite/blob/b58d5842/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
index f57d573..2a83941 100644
--- a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
+++ b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
@@ -632,7 +632,7 @@ public abstract class H2IndexingAbstractGeoSelfTest extends GridCacheAbstractSel
      */
     private static class Enemy {
         /** */
-        @QuerySqlField
+        @QuerySqlField(index = true)
         int campId;
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b58d5842/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 bf0276a..798ca9b 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
@@ -1569,7 +1569,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         Connection c = connectionForSpace(space);
 
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
-        final boolean distributedJoins = qry.isDistributedJoins() && cctx.isPartitioned();
+        final boolean distributedJoins = qry.isDistributedJoins();
         final boolean grpByCollocated = qry.isCollocated();
 
         final DistributedJoinMode distributedJoinMode = distributedJoinMode(qry.isLocal(), distributedJoins);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b58d5842/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 4578171..fa4eded 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -290,6 +290,35 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     * @throws InterruptedException If failed.
+     */
+    public void testDistributedJoinFromReplicatedCache() throws InterruptedException {
+        CacheConfiguration ccfg1 = cacheConfig("pers", true,
+            Integer.class, Person2.class);
+
+        CacheConfiguration ccfg2 = cacheConfig("org", true,
+            Integer.class, Organization.class);
+
+        CacheConfiguration ccfg3 = cacheConfig("orgRepl", false,
+            Integer.class, Organization.class);
+
+        IgniteCache<Integer, Person2> c1 = ignite(0).getOrCreateCache(ccfg1);
+        IgniteCache<Integer, Organization> c2 = ignite(0).getOrCreateCache(ccfg2);
+        IgniteCache<Integer, Organization> c3 = ignite(0).getOrCreateCache(ccfg3);
+
+        try {
+            awaitPartitionMapExchange();
+
+            doTestDistributedJoins(c3, c1, c2, 300, 2000, 5, false);
+            doTestDistributedJoins(c3, c1, c2, 300, 2000, 5, true);
+        }
+        finally {
+            c1.destroy();
+            c2.destroy();
+        }
+    }
+
     @SuppressWarnings("SuspiciousMethodCalls")
     public void testExists() {
         IgniteCache<Integer,Person2> x = ignite(0).getOrCreateCache(cacheConfig("x", true,
@@ -523,14 +552,14 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
         try {
             awaitPartitionMapExchange();
 
-            doTestDistributedJoins(c1, c2, 30, 100, 1000, false);
-            doTestDistributedJoins(c1, c2, 30, 100, 1000, true);
+            doTestDistributedJoins(c2, c1, c2, 30, 100, 1000, false);
+            doTestDistributedJoins(c2, c1, c2, 30, 100, 1000, true);
 
-            doTestDistributedJoins(c1, c2, 3, 10, 3, false);
-            doTestDistributedJoins(c1, c2, 3, 10, 3, true);
+            doTestDistributedJoins(c2, c1, c2, 3, 10, 3, false);
+            doTestDistributedJoins(c2, c1, c2, 3, 10, 3, true);
 
-            doTestDistributedJoins(c1, c2, 300, 2000, 5, false);
-            doTestDistributedJoins(c1, c2, 300, 2000, 5, true);
+            doTestDistributedJoins(c2, c1, c2, 300, 2000, 5, false);
+            doTestDistributedJoins(c2, c1, c2, 300, 2000, 5, true);
         }
         finally {
             c1.destroy();
@@ -1409,6 +1438,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
      * @param enforceJoinOrder Enforce join order.
      */
     private void doTestDistributedJoins(
+        IgniteCache<?,?> qryCache,
         IgniteCache<Integer, Person2> c1,
         IgniteCache<Integer, Organization> c2,
         int orgs,
@@ -1442,7 +1472,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
 
         String select = "select count(*) from \"org\".Organization o, \"pers\".Person2 p where p.orgId = o._key";
 
-        String plan = (String)c2.query(new SqlFieldsQuery("explain " + select)
+        String plan = (String)qryCache.query(new SqlFieldsQuery("explain " + select)
             .setDistributedJoins(true).setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize))
             .getAll().get(0).get(0);
 
@@ -1453,7 +1483,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
         else
             assertTrue(plan, plan.contains("batched:unicast"));
 
-        assertEquals(Long.valueOf(persons), c2.query(new SqlFieldsQuery(select).setDistributedJoins(true)
+        assertEquals((long)persons, qryCache.query(new SqlFieldsQuery(select).setDistributedJoins(true)
             .setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize)).getAll().get(0).get(0));
 
         c1.clear();