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 2017/05/31 09:22:56 UTC

[06/51] ignite git commit: IGNITE-5312: Changed cache name to schema name in H2TwoStepCachedQueryKey. This closes #2020.

IGNITE-5312: Changed cache name to schema name in H2TwoStepCachedQueryKey. This closes #2020.


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

Branch: refs/heads/ignite-5075-pds
Commit: 858e5b729b621efef057c1919273463ebdcb133d
Parents: e59d389
Author: devozerov <vo...@gridgain.com>
Authored: Sat May 27 21:07:54 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Sat May 27 21:07:54 2017 +0300

----------------------------------------------------------------------
 .../query/h2/H2TwoStepCachedQuery.java           | 18 ++++++++++++++++--
 .../query/h2/H2TwoStepCachedQueryKey.java        | 19 ++++++-------------
 .../processors/query/h2/IgniteH2Indexing.java    | 19 +++++++++++--------
 3 files changed, 33 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/858e5b72/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
index dd1b44c..6139869 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
@@ -28,10 +28,10 @@ import java.util.List;
  */
 public class H2TwoStepCachedQuery {
     /** */
-    final List<GridQueryFieldMetadata> meta;
+    private final List<GridQueryFieldMetadata> meta;
 
     /** */
-    final GridCacheTwoStepQuery twoStepQry;
+    private final GridCacheTwoStepQuery twoStepQry;
 
     /**
      * @param meta Fields metadata.
@@ -42,6 +42,20 @@ public class H2TwoStepCachedQuery {
         this.twoStepQry = twoStepQry;
     }
 
+    /**
+     * @return Fields metadata.
+     */
+    public List<GridQueryFieldMetadata> meta() {
+        return meta;
+    }
+
+    /**
+     * @return Query.
+     */
+    public GridCacheTwoStepQuery query() {
+        return twoStepQry;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(H2TwoStepCachedQuery.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/858e5b72/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
index 1452a83..cc14abd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
@@ -22,7 +22,7 @@ package org.apache.ignite.internal.processors.query.h2;
  */
 public class H2TwoStepCachedQueryKey {
     /** */
-    private final String cacheName;
+    private final String schemaName;
 
     /** */
     private final String sql;
@@ -40,20 +40,20 @@ public class H2TwoStepCachedQueryKey {
     private final boolean isLocal;
 
     /**
-     * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param sql Sql.
      * @param grpByCollocated Collocated GROUP BY.
      * @param distributedJoins Distributed joins enabled.
      * @param enforceJoinOrder Enforce join order of tables.
      * @param isLocal Query is local flag.
      */
-    H2TwoStepCachedQueryKey(String cacheName,
+    H2TwoStepCachedQueryKey(String schemaName,
         String sql,
         boolean grpByCollocated,
         boolean distributedJoins,
         boolean enforceJoinOrder,
         boolean isLocal) {
-        this.cacheName = cacheName;
+        this.schemaName = schemaName;
         this.sql = sql;
         this.grpByCollocated = grpByCollocated;
         this.distributedJoins = distributedJoins;
@@ -61,13 +61,6 @@ public class H2TwoStepCachedQueryKey {
         this.isLocal = isLocal;
     }
 
-    /**
-     * @return Cache name.
-     */
-    public String cacheName() {
-        return cacheName;
-    }
-
     /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
@@ -87,7 +80,7 @@ public class H2TwoStepCachedQueryKey {
         if (enforceJoinOrder != that.enforceJoinOrder)
             return false;
 
-        if (cacheName != null ? !cacheName.equals(that.cacheName) : that.cacheName != null)
+        if (schemaName != null ? !schemaName.equals(that.schemaName) : that.schemaName != null)
             return false;
 
         return isLocal == that.isLocal && sql.equals(that.sql);
@@ -95,7 +88,7 @@ public class H2TwoStepCachedQueryKey {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        int res = cacheName != null ? cacheName.hashCode() : 0;
+        int res = schemaName != null ? schemaName.hashCode() : 0;
         res = 31 * res + sql.hashCode();
         res = 31 * res + (grpByCollocated ? 1 : 0);
         res = res + (distributedJoins ? 2 : 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/858e5b72/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 0291b0a..c94c215 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
@@ -1324,12 +1324,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx,
         SqlFieldsQuery qry, boolean keepBinary, GridQueryCancel cancel) {
-        final String cacheName = cctx.name();
         final String sqlQry = qry.getSql();
 
-        String schema = schema(cctx.name());
+        String schemaName = schema(cctx.name());
 
-        Connection c = connectionForSchema(schema);
+        Connection c = connectionForSchema(schemaName);
 
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
         final boolean distributedJoins = qry.isDistributedJoins();
@@ -1340,13 +1339,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridCacheTwoStepQuery twoStepQry = null;
         List<GridQueryFieldMetadata> meta;
 
-        final H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(cacheName, sqlQry, grpByCollocated,
+        final H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, sqlQry, grpByCollocated,
             distributedJoins, enforceJoinOrder, qry.isLocal());
         H2TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
         if (cachedQry != null) {
-            twoStepQry = cachedQry.twoStepQry.copy();
-            meta = cachedQry.meta;
+            twoStepQry = cachedQry.query().copy();
+            meta = cachedQry.meta();
         }
         else {
             final UUID locNodeId = ctx.localNodeId();
@@ -1415,7 +1414,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 if (twoStepQry == null) {
                     if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                         try {
-                            return dmlProc.updateSqlFieldsDistributed(schema, stmt, qry, cancel);
+                            return dmlProc.updateSqlFieldsDistributed(schemaName, stmt, qry, cancel);
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DML statement [stmt=" + sqlQry +
@@ -2249,11 +2248,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 for (Index idx : tblDesc.table().getIndexes())
                     idx.close(null);
 
+            int cacheId = CU.cacheId(cacheName);
+
             for (Iterator<Map.Entry<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery>> it =
                 twoStepCache.entrySet().iterator(); it.hasNext();) {
                 Map.Entry<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery> e = it.next();
 
-                if (F.eq(e.getKey().cacheName(), cacheName))
+                GridCacheTwoStepQuery qry = e.getValue().query();
+
+                if (!F.isEmpty(qry.cacheIds()) && qry.cacheIds().contains(cacheId))
                     it.remove();
             }
         }