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 2015/11/24 11:31:01 UTC

[5/8] ignite git commit: ignite-sql-cache-stmt

ignite-sql-cache-stmt


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

Branch: refs/heads/ignite-sql-opt-2
Commit: ab077e3e5b9af951b4d5b4884235f0452252faf0
Parents: 5e000f8
Author: sboikov <sb...@gridgain.com>
Authored: Tue Nov 24 10:35:56 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Nov 24 10:35:56 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   | 56 +++++++++++++-------
 .../h2/twostep/GridReduceQueryExecutor.java     |  1 -
 2 files changed, 38 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab077e3e/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 4ee1773..3fbbe36 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
@@ -93,6 +93,7 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridLuceneIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
@@ -125,7 +126,6 @@ import org.h2.table.Column;
 import org.h2.table.IndexColumn;
 import org.h2.table.Table;
 import org.h2.tools.Server;
-import org.h2.util.SmallLRUCache;
 import org.h2.util.Utils;
 import org.h2.value.DataType;
 import org.h2.value.Value;
@@ -178,6 +178,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0;" +
         "RECOMPILE_ALWAYS=1;MAX_OPERATION_MEMORY=0";
 
+    /** */
+    private static final int PREPARED_STMT_CACHE_SIZE = 256;
+
+    /** */
+    private static final int TWO_STEP_QRY_CACHE_SIZE = 256;
+
     /** Field name for key. */
     public static final String KEY_FIELD_NAME = "_key";
 
@@ -272,14 +278,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final ThreadLocal<StatementCache<String, PreparedStatement>> stmtCache =
         new ThreadLocal<StatementCache<String, PreparedStatement>>() {
             @Override protected StatementCache<String, PreparedStatement> initialValue() {
-                return new StatementCache(256);
+                return new StatementCache<>(PREPARED_STMT_CACHE_SIZE);
             }
         };
 
-    /**
-     * TODO add some kind of eviction like LRU
-     */
-    private final ConcurrentMap<T2<String, String>, TwoStepCachedQuery> twoStepCache = new ConcurrentHashMap8<>();
+    /** */
+    private final GridBoundedConcurrentLinkedHashMap<T2<String, String>, TwoStepCachedQuery> twoStepCache =
+        new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
 
     /**
      * @param space Space.
@@ -991,7 +996,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
         if (cachedQry != null) {
-            twoStepQry = cachedQry.twoStepQuery.copy(qry.getArgs());
+            twoStepQry = cachedQry.twoStepQry.copy(qry.getArgs());
             meta = cachedQry.meta;
         }
         else {
@@ -1032,9 +1037,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         cursor.fieldsMeta(meta);
 
         if (cachedQry == null && !twoStepQry.explain()) {
-            cachedQry = new TwoStepCachedQuery();
-            cachedQry.meta = meta;
-            cachedQry.twoStepQuery = twoStepQry.copy(null);
+            cachedQry = new TwoStepCachedQuery(meta, twoStepQry.copy(null));
             twoStepCache.putIfAbsent(cachedQryKey, cachedQry);
         }
 
@@ -1042,14 +1045,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Cached two-step query.
-     */
-    private static final class TwoStepCachedQuery {
-        List<GridQueryFieldMetadata> meta;
-        GridCacheTwoStepQuery twoStepQuery;
-    }
-
-    /**
      * Sets filters for current thread. Must be set to not null value
      * before executeQuery and reset to null after in finally block since it signals
      * to table that it should return content without expired values.
@@ -1618,6 +1613,31 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * Cached two-step query.
+     */
+    private static final class TwoStepCachedQuery {
+        /** */
+        final List<GridQueryFieldMetadata> meta;
+
+        /** */
+        final GridCacheTwoStepQuery twoStepQry;
+
+        /**
+         * @param meta Fields metadata.
+         * @param twoStepQry Query.
+         */
+        public TwoStepCachedQuery(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
+            this.meta = meta;
+            this.twoStepQry = twoStepQry;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TwoStepCachedQuery.class, this);
+        }
+    }
+
+    /**
      * Wrapper to store connection and flag is schema set or not.
      */
     private static class ConnectionWrapper {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab077e3e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index f5868cc..78c1f77 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.h2.twostep;
 
 import java.lang.reflect.Constructor;
 import java.sql.Connection;
-import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;