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/12/22 15:15:45 UTC

[05/50] [abbrv] ignite git commit: IGNITE-4379: Fixed local query execution. This closes #1323.

IGNITE-4379: Fixed local query execution. This closes #1323.


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

Branch: refs/heads/master
Commit: 6e485637e2738a7e809eac1a802f0964dc12383d
Parents: b83ec8e
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Dec 9 15:42:40 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 15:42:40 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   | 54 ++++++++++----------
 .../IgniteCacheAbstractFieldsQuerySelfTest.java | 19 +++++++
 2 files changed, 45 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6e485637/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 6da8758..362ddd8 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
@@ -774,31 +774,38 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         throws IgniteCheckedException {
         final Connection conn = connectionForSpace(spaceName);
 
-        initLocalQueryContext(conn, enforceJoinOrder, filters);
+        setupConnection(conn, false, enforceJoinOrder);
+
+        final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
+
+        List<GridQueryFieldMetadata> meta;
 
         try {
-            final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
+            meta = meta(stmt.getMetaData());
+        }
+        catch (SQLException e) {
+            throw new IgniteCheckedException("Cannot prepare query metadata", e);
+        }
 
-            List<GridQueryFieldMetadata> meta;
+        final GridH2QueryContext ctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL)
+            .filter(filters).distributedJoins(false);
 
-            try {
-                meta = meta(stmt.getMetaData());
-            }
-            catch (SQLException e) {
-                throw new IgniteCheckedException("Cannot prepare query metadata", e);
-            }
+        return new GridQueryFieldsResultAdapter(meta, null) {
+            @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException {
+                assert GridH2QueryContext.get() == null;
+
+                GridH2QueryContext.set(ctx);
 
-            return new GridQueryFieldsResultAdapter(meta, null) {
-                @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException{
+                try {
                     ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
 
                     return new FieldsIterator(rs);
                 }
-            };
-        }
-        finally {
-            GridH2QueryContext.clearThreadLocal();
-        }
+                finally {
+                    GridH2QueryContext.clearThreadLocal();
+                }
+            }
+        };
     }
 
     /**
@@ -1025,17 +1032,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param conn Connection.
-     * @param enforceJoinOrder Enforce join order of tables.
-     * @param filter Filter.
-     */
-    private void initLocalQueryContext(Connection conn, boolean enforceJoinOrder, IndexingQueryFilter filter) {
-        setupConnection(conn, false, enforceJoinOrder);
-
-        GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false));
-    }
-
-    /**
      * @param conn Connection to use.
      * @param distributedJoins If distributed joins are enabled.
      * @param enforceJoinOrder Enforce join order of tables.
@@ -1061,7 +1057,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         Connection conn = connectionForThread(tbl.schemaName());
 
-        initLocalQueryContext(conn, false, filter);
+        setupConnection(conn, false, false);
+
+        GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false));
 
         try {
             ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6e485637/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index d5f02eb..2c35501 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -28,10 +28,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.AffinityKey;
@@ -650,6 +652,23 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
     }
 
     /** @throws Exception If failed. */
+    public void testLocalQuery() throws Exception {
+        IgniteCache<Object, Object> cache = grid(0).cache( null);
+
+        int expected = 0;
+
+        for(Cache.Entry e: cache.localEntries(CachePeekMode.PRIMARY)){
+            if(e.getValue() instanceof Integer)
+                expected++;
+        }
+
+        QueryCursor<List<?>> qry = cache
+            .query(new SqlFieldsQuery("select _key, _val from Integer").setLocal(true));
+
+        assertEquals(expected, qry.getAll().size());
+    }
+
+    /** @throws Exception If failed. */
     public void testQueryIntegersWithJoin() throws Exception {
         QueryCursor<List<?>> qry = grid(0).cache(null).query(new SqlFieldsQuery(
             "select i._KEY, i._VAL, j._KEY, j._VAL from Integer i join Integer j where i._VAL >= 100"));