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/05/12 19:20:07 UTC

[48/50] [abbrv] incubator-ignite git commit: ignite-745 Query metrics do not updated for SQL queries

ignite-745 Query metrics do not updated for SQL queries


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

Branch: refs/heads/ignite-745
Commit: f6eaaad3b1721e70a39a09d82388605270bd4875
Parents: 211754d
Author: agura <ag...@gridgain.com>
Authored: Tue May 12 19:39:14 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue May 12 19:39:14 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/query/QueryMetrics.java |   7 +-
 .../cache/query/GridCacheQueryAdapter.java      |  14 +-
 .../processors/query/GridQueryProcessor.java    | 206 ++++++++++++-------
 3 files changed, 140 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6eaaad3/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
index 1d1eed1..e8f107c 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
@@ -17,12 +17,11 @@
 
 package org.apache.ignite.cache.query;
 
-import org.apache.ignite.internal.processors.cache.query.*;
-
 /**
  * Cache query metrics used to obtain statistics on query. You can get metrics for
- * particular query via {@link org.apache.ignite.internal.processors.cache.query.CacheQuery#metrics()} method or accumulated metrics
- * for all queries via {@link org.apache.ignite.internal.processors.cache.query.CacheQueries#metrics()}.
+ * particular query via {@link org.apache.ignite.internal.processors.cache.query.CacheQuery#metrics()} method
+ * or accumulated metrics for all queries via
+ * {@link org.apache.ignite.internal.processors.cache.query.CacheQueries#metrics()}.
  */
 public interface QueryMetrics {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6eaaad3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 4b1fc87..36c9fcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -348,17 +348,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
      * @param duration Duration.
      */
     public void onExecuted(Object res, Throwable err, long startTime, long duration) {
-        boolean fail = err != null;
-
-        // Update own metrics.
-        metrics.onQueryExecute(duration, fail);
-
-        // Update metrics in query manager.
-        cctx.queries().onMetricsUpdate(duration, fail);
-
-        if (log.isDebugEnabled())
-            log.debug("Query execution finished [qry=" + this + ", startTime=" + startTime +
-                ", duration=" + duration + ", fail=" + fail + ", res=" + res + ']');
+        GridQueryProcessor.onExecuted(cctx, metrics, res, err, startTime, duration, log);
     }
 
     /** {@inheritDoc} */
@@ -376,10 +366,12 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         return execute(null, rmtTransform, args);
     }
 
+    /** {@inheritDoc} */
     @Override public QueryMetrics metrics() {
         return metrics.copy();
     }
 
+    /** {@inheritDoc} */
     @Override public void resetMetrics() {
         metrics = new GridCacheQueryMetricsAdapter();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6eaaad3/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 7ce894d..2839173 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -562,59 +562,63 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public <K,V> Iterator<Cache.Entry<K,V>> queryLocal(GridCacheContext<?,?> cctx, SqlQuery qry) {
+    public <K,V> Iterator<Cache.Entry<K,V>> queryLocal(final GridCacheContext<?,?> cctx, final SqlQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            String space = cctx.name();
-            String type = qry.getType();
-            String sqlQry = qry.getSql();
-            Object[] params = qry.getArgs();
-
-            TypeDescriptor typeDesc = typesByName.get(new TypeName(space, type));
-
-            if (typeDesc == null || !typeDesc.registered())
-                throw new CacheException("Failed to find SQL table for type: " + type);
-
-            final GridCloseableIterator<IgniteBiTuple<K,V>> i = idx.query(space, sqlQry, F.asList(params), typeDesc,
-                idx.backupFilter());
-
-            if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
-                ctx.event().record(new CacheQueryExecutedEvent<>(
-                    ctx.discovery().localNode(),
-                    "SQL query executed.",
-                    EVT_CACHE_QUERY_EXECUTED,
-                    CacheQueryType.SQL.name(),
-                    null,
-                    null,
-                    sqlQry,
-                    null,
-                    null,
-                    params,
-                    null,
-                    null));
-            }
+            return executeQuery(cctx, new IgniteOutClosureX<Iterator<Cache.Entry<K, V>>>() {
+                @Override public Iterator<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
+                    String space = cctx.name();
+                    String type = qry.getType();
+                    String sqlQry = qry.getSql();
+                    Object[] params = qry.getArgs();
+
+                    TypeDescriptor typeDesc = typesByName.get(new TypeName(space, type));
+
+                    if (typeDesc == null || !typeDesc.registered())
+                        throw new CacheException("Failed to find SQL table for type: " + type);
+
+                    final GridCloseableIterator<IgniteBiTuple<K,V>> i = idx.query(space, sqlQry, F.asList(params),
+                        typeDesc, idx.backupFilter());
+
+                    if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
+                        ctx.event().record(new CacheQueryExecutedEvent<>(
+                            ctx.discovery().localNode(),
+                            "SQL query executed.",
+                            EVT_CACHE_QUERY_EXECUTED,
+                            CacheQueryType.SQL.name(),
+                            null,
+                            null,
+                            sqlQry,
+                            null,
+                            null,
+                            params,
+                            null,
+                            null));
+                    }
 
-            return new ClIter<Cache.Entry<K,V>>() {
-                @Override public void close() throws Exception {
-                    i.close();
-                }
+                    return new ClIter<Cache.Entry<K,V>>() {
+                        @Override public void close() throws Exception {
+                            i.close();
+                        }
 
-                @Override public boolean hasNext() {
-                    return i.hasNext();
-                }
+                        @Override public boolean hasNext() {
+                            return i.hasNext();
+                        }
 
-                @Override public Cache.Entry<K,V> next() {
-                    IgniteBiTuple<K,V> t = i.next();
+                        @Override public Cache.Entry<K,V> next() {
+                            IgniteBiTuple<K,V> t = i.next();
 
-                    return new CacheEntryImpl<>(t.getKey(), t.getValue());
-                }
+                            return new CacheEntryImpl<>(t.getKey(), t.getValue());
+                        }
 
-                @Override public void remove() {
-                    throw new UnsupportedOperationException();
+                        @Override public void remove() {
+                            throw new UnsupportedOperationException();
+                        }
+                    };
                 }
-            };
+            });
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);
@@ -636,39 +640,43 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Iterator.
      */
-    public QueryCursor<List<?>> queryLocalFields(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            String space = cctx.name();
-            String sql = qry.getSql();
-            Object[] args = qry.getArgs();
-
-            GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
-
-            if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
-                ctx.event().record(new CacheQueryExecutedEvent<>(
-                        ctx.discovery().localNode(),
-                        "SQL query executed.",
-                        EVT_CACHE_QUERY_EXECUTED,
-                        CacheQueryType.SQL.name(),
-                        null,
-                        null,
-                        sql,
-                        null,
-                        null,
-                        args,
-                        null,
-                        null));
-            }
+            return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                    String space = cctx.name();
+                    String sql = qry.getSql();
+                    Object[] args = qry.getArgs();
+
+                    GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
+
+                    if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
+                        ctx.event().record(new CacheQueryExecutedEvent<>(
+                            ctx.discovery().localNode(),
+                            "SQL query executed.",
+                            EVT_CACHE_QUERY_EXECUTED,
+                            CacheQueryType.SQL.name(),
+                            null,
+                            null,
+                            sql,
+                            null,
+                            null,
+                            args,
+                            null,
+                            null));
+                    }
 
-            QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-                new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable()));
+                    QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
+                        new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable()));
 
-            cursor.fieldsMeta(res.metaData());
+                    cursor.fieldsMeta(res.metaData());
 
-            return cursor;
+                    return cursor;
+                }
+            });
         }
         catch (IgniteCheckedException e) {
             throw new CacheException(e);
@@ -770,7 +778,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Field rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params,
+    public GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params,
         IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
@@ -1291,9 +1299,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         try {
-            Method method = dataTypeCls.getMethod("isGeometryClass", Class.class);
+            Method mtd = dataTypeCls.getMethod("isGeometryClass", Class.class);
 
-            return (Boolean)method.invoke(null, cls);
+            return (Boolean)mtd.invoke(null, cls);
         }
         catch (Exception e) {
             throw new IgniteCheckedException("Failed to invoke 'org.h2.value.DataType.isGeometryClass' method.", e);
@@ -1301,6 +1309,60 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cctx Cache context.
+     * @param clo Closure.
+     */
+    private <R> R executeQuery(GridCacheContext<?,?> cctx, IgniteOutClosureX<R> clo)
+        throws IgniteCheckedException {
+        final long start = U.currentTimeMillis();
+
+        Throwable err = null;
+        
+        R res = null;
+
+        try {
+            res = clo.apply();
+            
+            return res;
+        }
+        catch (GridClosureException e) {
+            err = e.unwrap();
+
+            throw (IgniteCheckedException)err;
+        }
+        finally {
+            GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.cache().queries().metrics();
+
+            onExecuted(cctx, metrics, res, err, start, U.currentTimeMillis() - start, log);
+        }
+    }
+
+    /**
+     * @param cctx Cctx.
+     * @param metrics Metrics.
+     * @param res Result.
+     * @param err Err.
+     * @param startTime Start time.
+     * @param duration Duration.
+     * @param log Logger.
+     */
+    public static void onExecuted(GridCacheContext<?, ?> cctx, GridCacheQueryMetricsAdapter metrics,
+        Object res, Throwable err, long startTime, long duration, IgniteLogger log) {
+        boolean fail = err != null;
+
+        // Update own metrics.
+        metrics.onQueryExecute(duration, fail);
+
+        // Update metrics in query manager.
+        cctx.queries().onMetricsUpdate(duration, fail);
+
+        if (log.isTraceEnabled())
+            log.trace("Query execution finished [startTime=" + startTime +
+                ", duration=" + duration + ", fail=" + (err != null) + ", res=" + res + ']');
+
+    }
+
+    /**
      *
      */
     private abstract static class Property {