You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2015/02/02 02:54:17 UTC

[49/50] [abbrv] incubator-ignite git commit: ignite-sql - renamings

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlField.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlField.java
deleted file mode 100644
index c5560a5..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlField.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import java.lang.annotation.*;
-
-/**
- * Annotates fields for SQL queries. All fields that will be involved in SQL clauses must have
- * this annotation. For more information about cache queries see {@link CacheQuery} documentation.
- * @see CacheQuery
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.METHOD, ElementType.FIELD})
-public @interface CacheQuerySqlField {
-    /**
-     * Specifies whether cache should maintain an index for this field or not.
-     * Just like with databases, field indexing may require additional overhead
-     * during updates, but makes select operations faster.
-     * <p>
-     * When indexing SPI and indexed field is
-     * of type {@code com.vividsolutions.jts.geom.Geometry} (or any subclass of this class) then GridGain will
-     * consider this index as spatial providing performance boost for spatial queries.
-     *
-     * @return {@code True} if index must be created for this field in database.
-     */
-    boolean index() default false;
-
-    /**
-     * Specifies whether index should be unique or not. This property only
-     * makes sense if {@link #index()} property is set to {@code true}.
-     *
-     * @return {@code True} if field index should be unique.
-     * @deprecated No longer supported, will be ignored.
-     */
-    @Deprecated
-    boolean unique() default false;
-
-    /**
-     * Specifies whether index should be in descending order or not. This property only
-     * makes sense if {@link #index()} property is set to {@code true}.
-     *
-     * @return {@code True} if field index should be in descending order.
-     */
-    boolean descending() default false;
-
-    /**
-     * Array of index groups this field belongs to. Groups are used for compound indexes,
-     * whenever index should be created on more than one field. All fields within the same
-     * group will belong to the same index.
-     * <p>
-     * Group indexes are needed because SQL engine can utilize only one index per table occurrence in a query.
-     * For example if we have two separate indexes on fields {@code a} and {@code b} of type {@code X} then
-     * query {@code select * from X where a = ? and b = ?} will use for filtering either index on field {@code a}
-     * or {@code b} but not both. For more effective query execution here it is preferable to have a single
-     * group index on both fields.
-     * <p>
-     * For more complex scenarios please refer to {@link CacheQuerySqlField.Group} documentation.
-     *
-     * @return Array of group names.
-     */
-    String[] groups() default {};
-
-    /**
-     * Array of ordered index groups this field belongs to. For more information please refer to
-     * {@linkplain CacheQuerySqlField.Group} documentation.
-     *
-     * @return Array of ordered group indexes.
-     * @see #groups()
-     */
-    Group[] orderedGroups() default {};
-
-    /**
-     * Property name. If not provided then field name will be used.
-     *
-     * @return Name of property.
-     */
-    String name() default "";
-
-    /**
-     * Describes group of index and position of field in this group.
-     * <p>
-     * Opposite to {@link #groups()} this annotation gives control over order of fields in a group index.
-     * This can be needed in scenarios when we have a query like
-     * {@code select * from X where a = ? and b = ? order by b desc}. If we have index {@code (a asc, b asc)}
-     * sorting on {@code b} will be performed. Here it is preferable to have index {@code (b desc, a asc)}
-     * which will still allow query to search on index using both fields and avoid sorting because index
-     * is already sorted in needed way.
-     *
-     * @see #groups()
-     * @see #orderedGroups()
-     */
-    @Retention(RetentionPolicy.RUNTIME)
-    @Target({ElementType.METHOD, ElementType.FIELD})
-    @SuppressWarnings("PublicInnerClass")
-    public static @interface Group {
-        /**
-         * Group index name where this field participate.
-         *
-         * @return Group index name
-         */
-        String name();
-
-        /**
-         * Fields in this group index will be sorted on this attribute.
-         *
-         * @return Order number.
-         */
-        int order();
-
-        /**
-         * Defines sorting order for this field in group.
-         *
-         * @return True if field will be in descending order.
-         */
-        boolean descending() default false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlFunction.java
deleted file mode 100644
index 2fd11f4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQuerySqlFunction.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import java.lang.annotation.*;
-
-/**
- * Annotates public static methods in classes to be used in SQL queries as custom functions.
- * Annotated class must be registered in H2 indexing SPI using following method
- * {@link org.apache.ignite.configuration.IgniteQueryConfiguration#setIndexCustomFunctionClasses(Class[])}.
- * <p>
- * Example usage:
- * <pre name="code" class="java">
- *     public class MyFunctions {
- *         &#64;CacheQuerySqlFunction
- *         public static int sqr(int x) {
- *             return x * x;
- *         }
- *     }
- *
- *     // Register.
- *     indexing.setIndexCustomFunctionClasses(MyFunctions.class);
- *
- *     // And use in queries.
- *     cache.queries().createSqlFieldsQuery("select sqr(2) where sqr(1) = 1");
- * </pre>
- * <p>
- * For more information about H2 custom functions please refer to
- * <a href="http://www.h2database.com/html/features.html#user_defined_functions">H2 documentation</a>.
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.METHOD)
-public @interface CacheQuerySqlFunction {
-    /**
-     * Specifies alias for the function to be used form SQL queries.
-     * If no alias provided method name will be used.
-     *
-     * @return Alias for function.
-     */
-    String alias() default "";
-
-    /**
-     * Specifies if the function is deterministic (result depends only on input parameters).
-     * <p>
-     * Deterministic function is a function which always returns the same result
-     * assuming that input parameters are the same.
-     *
-     * @return {@code true} If function is deterministic, {@code false} otherwise.
-     */
-    boolean deterministic() default false;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTextField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTextField.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTextField.java
deleted file mode 100644
index d48cc23..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTextField.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import java.lang.annotation.*;
-
-/**
- * Annotation for fields or getters to be indexed for full text
- * search using {@code H2 TEXT} indexing. For more information
- * refer to {@link CacheQuery} documentation.
- * @see CacheQuery
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.METHOD, ElementType.FIELD, ElementType.TYPE})
-public @interface CacheQueryTextField {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryType.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryType.java
deleted file mode 100644
index a418cfd..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryType.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-/**
- * Cache query type.
- * <p>
- * Used in {@link org.apache.ignite.events.IgniteCacheQueryExecutedEvent} and {@link org.apache.ignite.events.IgniteCacheQueryReadEvent}
- * to identify the type of query for which an event was fired.
- *
- * @see org.apache.ignite.events.IgniteCacheQueryExecutedEvent#queryType()
- * @see org.apache.ignite.events.IgniteCacheQueryReadEvent#queryType()
- */
-public enum CacheQueryType {
-    /** SQL query. */
-    SQL,
-
-    /** SQL fields query. */
-    SQL_FIELDS,
-
-    /** Full text query. */
-    FULL_TEXT,
-
-    /** Scan query. */
-    SCAN,
-
-    /** Continuous query. */
-    CONTINUOUS,
-
-    /** SPI query. */
-    SPI
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeMetadata.java
deleted file mode 100644
index 7b0afff..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeMetadata.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-import org.apache.ignite.lang.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.*;
-
-/**
- * Cache query type metadata.
- */
-public class CacheQueryTypeMetadata {
-    /** Type name, e.g. class name. */
-    @GridToStringInclude
-    private String type;
-
-    /** Fields to be queried, in addition to indexed fields. */
-    @GridToStringInclude
-    private Map<String, Class<?>> qryFlds = new HashMap<>();
-
-    /** Fields to index in ascending order. */
-    @GridToStringInclude
-    private Map<String, Class<?>> ascFlds = new HashMap<>();
-
-    /** Fields to index in descending order. */
-    @GridToStringInclude
-    private Map<String, Class<?>> descFlds = new HashMap<>();
-
-    /** Fields to index as text. */
-    @GridToStringInclude
-    private Collection<String> txtFlds = new LinkedHashSet<>();
-
-    /** Fields to create group indexes for. */
-    @GridToStringInclude
-    private Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps;
-
-    /**
-     * Default constructor.
-     */
-    public CacheQueryTypeMetadata() {
-        // No-op.
-    }
-
-    /**
-     *
-     */
-    public CacheQueryTypeMetadata(CacheQueryTypeMetadata src) {
-        type = src.getType();
-
-        qryFlds = new HashMap<>(src.getQueryFields());
-        ascFlds = new HashMap<>(src.getAscendingFields());
-        descFlds = new HashMap<>(src.getDescendingFields());
-        txtFlds = new HashSet<>(src.getTextFields());
-
-        grps = new HashMap<>(src.getGroups());
-    }
-
-    /**
-     * Gets type (e.g. class name).
-     *
-     * @return Type name.
-     */
-    public String getType() {
-        return type;
-    }
-
-    /**
-     * Sets type.
-     *
-     * @param cls Type class.
-     */
-    public void setType(Class<?> cls) {
-        setType(cls.getName());
-    }
-
-    /**
-     * Sets type.
-     *
-     * @param type Type name.
-     */
-    public void setType(String type) {
-        this.type = type;
-    }
-
-    /**
-     * Gets query-enabled fields.
-     *
-     * @return Collection of fields available for query.
-     */
-    public Map<String, Class<?>> getQueryFields() {
-        return qryFlds;
-    }
-
-    /**
-     * Sets query fields map.
-     *
-     * @param qryFlds Query fields.
-     */
-    public void setQueryFields(Map<String, Class<?>> qryFlds) {
-        this.qryFlds = qryFlds;
-    }
-
-    /**
-     * Gets ascending-indexed fields.
-     *
-     * @return Map of ascending-indexed fields.
-     */
-    public Map<String, Class<?>> getAscendingFields() {
-        return ascFlds;
-    }
-
-    /**
-     * Sets ascending-indexed fields.
-     *
-     * @param ascFlds Map of ascending-indexed fields.
-     */
-    public void setAscendingFields(Map<String, Class<?>> ascFlds) {
-        this.ascFlds = ascFlds;
-    }
-
-    /**
-     * Gets descending-indexed fields.
-     *
-     * @return Map of descending-indexed fields.
-     */
-    public Map<String, Class<?>> getDescendingFields() {
-        return descFlds;
-    }
-
-    /**
-     * Sets descending-indexed fields.
-     *
-     * @param descFlds Map of descending-indexed fields.
-     */
-    public void setDescendingFields(Map<String, Class<?>> descFlds) {
-        this.descFlds = descFlds;
-    }
-
-    /**
-     * Gets text-indexed fields.
-     *
-     * @return Collection of text indexed fields.
-     */
-    public Collection<String> getTextFields() {
-        return txtFlds;
-    }
-
-    /**
-     * Sets text-indexed fields.
-     *
-     * @param txtFlds Text-indexed fields.
-     */
-    public void setTextFields(Collection<String> txtFlds) {
-        this.txtFlds = txtFlds;
-    }
-
-    /**
-     * Gets group-indexed fields.
-     *
-     * @return Map of group-indexed fields.
-     */
-    public Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> getGroups() {
-        return grps;
-    }
-
-    /**
-     * Sets group-indexed fields.
-     *
-     * @param grps Map of group-indexed fields from index name to index fields.
-     */
-    public void setGroups(Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps) {
-        this.grps = grps;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(CacheQueryTypeMetadata.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeResolver.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeResolver.java
deleted file mode 100644
index 9fea55e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryTypeResolver.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.query;
-
-/**
- * Interface allowing to override table name for portable objects stored in cache.
- */
-public interface CacheQueryTypeResolver {
-    /**
-     * Allows to override type name for portable objects being stored in cache.
-     *
-     * @param key Key.
-     * @param val Value.
-     * @return Type name.
-     */
-    public String resolveTypeName(Object key, Object val);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/QueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryConfiguration.java
new file mode 100644
index 0000000..960c729
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryConfiguration.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Query configuration object.
+ */
+public class QueryConfiguration implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Collection of query type metadata. */
+    private Collection<QueryTypeMetadata> typeMeta;
+
+    /** Query type resolver. */
+    private QueryTypeResolver typeRslvr;
+
+    /** */
+    private boolean idxPrimitiveKey;
+
+    /** */
+    private boolean idxPrimitiveVal;
+
+    /** */
+    private boolean idxFixedTyping;
+
+    /** */
+    private boolean escapeAll;
+
+    /**
+     * Default constructor.
+     */
+    public QueryConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * @param cfg Configuration to copy.
+     */
+    public QueryConfiguration(QueryConfiguration cfg) {
+        typeMeta = cfg.getTypeMetadata();
+        typeRslvr = cfg.getTypeResolver();
+        idxPrimitiveKey = cfg.isIndexPrimitiveKey();
+        idxPrimitiveVal = cfg.isIndexPrimitiveValue();
+        idxFixedTyping = cfg.isIndexFixedTyping();
+        escapeAll = cfg.isEscapeAll();
+    }
+
+    /**
+     * Gets collection of query type metadata objects.
+     *
+     * @return Collection of query type metadata.
+     */
+    public Collection<QueryTypeMetadata> getTypeMetadata() {
+        return typeMeta;
+    }
+
+    /**
+     * Sets collection of query type metadata objects.
+     *
+     * @param typeMeta Collection of query type metadata.
+     */
+    public void setTypeMetadata(Collection<QueryTypeMetadata> typeMeta) {
+        this.typeMeta = typeMeta;
+    }
+
+    /**
+     * Gets query type resolver.
+     *
+     * @return Query type resolver.
+     */
+    public QueryTypeResolver getTypeResolver() {
+        return typeRslvr;
+    }
+
+    /**
+     * Sets query type resolver.
+     *
+     * @param typeRslvr Query type resolver.
+     */
+    public void setTypeResolver(QueryTypeResolver typeRslvr) {
+        this.typeRslvr = typeRslvr;
+    }
+
+    /**
+     * Gets flag indicating whether SQL engine should index by key in cases
+     * where key is primitive type
+     *
+     * @return {@code True} if primitive keys should be indexed.
+     */
+    public boolean isIndexPrimitiveKey() {
+        return idxPrimitiveKey;
+    }
+
+    /**
+     * Sets flag indicating whether SQL engine should index by key in cases
+     * where key is primitive type.
+     *
+     * @param idxPrimitiveKey {@code True} if primitive keys should be indexed.
+     */
+    public void setIndexPrimitiveKey(boolean idxPrimitiveKey) {
+        this.idxPrimitiveKey = idxPrimitiveKey;
+    }
+
+    /**
+     * Gets flag indicating whether SQL engine should index by value in cases
+     * where value is primitive type
+     *
+     * @return {@code True} if primitive values should be indexed.
+     */
+    public boolean isIndexPrimitiveValue() {
+        return idxPrimitiveVal;
+    }
+
+    /**
+     * Sets flag indexing whether SQL engine should index by value in cases
+     * where value is primitive type.
+     *
+     * @param idxPrimitiveVal {@code True} if primitive values should be indexed.
+     */
+    public void setIndexPrimitiveValue(boolean idxPrimitiveVal) {
+        this.idxPrimitiveVal = idxPrimitiveVal;
+    }
+
+    /**
+     * This flag essentially controls whether all values of the same type have
+     * identical key type.
+     * <p>
+     * If {@code false}, SQL engine will store all keys in BINARY form to make it possible to store
+     * the same value type with different key types. If {@code true}, key type will be converted
+     * to respective SQL type if it is possible, hence, improving performance of queries.
+     * <p>
+     * Setting this value to {@code false} also means that {@code '_key'} column cannot be indexed and
+     * cannot participate in query where clauses. The behavior of using '_key' column in where
+     * clauses with this flag set to {@code false} is undefined.
+     *
+     * @return {@code True} if SQL engine should try to convert values to their respective SQL
+     *      types for better performance.
+     */
+    public boolean isIndexFixedTyping() {
+        return idxFixedTyping;
+    }
+
+    /**
+     * This flag essentially controls whether key type is going to be identical
+     * for all values of the same type.
+     * <p>
+     * If false, SQL engine will store all keys in BINARY form to make it possible to store
+     * the same value type with different key types. If true, key type will be converted
+     * to respective SQL type if it is possible, which may provide significant performance
+     * boost.
+     *
+     * @param idxFixedTyping {@code True} if SQL engine should try to convert values to their respective SQL
+     *      types for better performance.
+     */
+    public void setIndexFixedTyping(boolean idxFixedTyping) {
+        this.idxFixedTyping = idxFixedTyping;
+    }
+
+    /**
+     * If {@code true}, then table name and all column names in 'create table' SQL
+     * generated for SQL engine are escaped with double quotes. This flag should be set if table name of
+     * column name is H2 reserved word or is not valid H2 identifier (e.g. contains space or hyphen).
+     * <p>
+     * Note if this flag is set then table and column name in SQL queries also must be escaped with double quotes.
+
+     * @return Flag value.
+     */
+    public boolean isEscapeAll() {
+        return escapeAll;
+    }
+
+    /**
+     * If {@code true}, then table name and all column names in 'create table' SQL
+     * generated for SQL engine are escaped with double quotes. This flag should be set if table name of
+     * column name is H2 reserved word or is not valid H2 identifier (e.g. contains space or hyphen).
+     * <p>
+     * Note if this flag is set then table and column name in SQL queries also must be escaped with double quotes.
+
+     * @param escapeAll Flag value.
+     */
+    public void setEscapeAll(boolean escapeAll) {
+        this.escapeAll = escapeAll;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
index eca11c3..ad96b35 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryContinuousPredicate.java
@@ -104,7 +104,7 @@ import javax.cache.event.*;
  * qry.cancel();
  * </pre>
  * Note that one query instance can be executed only once. After it's cancelled, it's non-operational.
- * If you need to repeat execution, use {@link CacheQueries#createContinuousQuery()} method to create
+ * If you need to repeat execution, use {@link org.apache.ignite.internal.processors.cache.query.CacheQueries#createContinuousQuery()} method to create
  * new query.
  */
 // TODO: make class.
@@ -197,7 +197,7 @@ public final class QueryContinuousPredicate<K, V> extends QueryPredicate<K, V> i
     /**
      * Stops continuous query execution. <p> Note that one query instance can be executed only once. After it's
      * cancelled, it's non-operational. If you need to repeat execution, use {@link
-     * CacheQueries#createContinuousQuery()} method to create new query.
+     * org.apache.ignite.internal.processors.cache.query.CacheQueries#createContinuousQuery()} method to create new query.
      *
      * @throws IgniteCheckedException In case of error.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/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
new file mode 100644
index 0000000..94e890c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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()}.
+ */
+public interface QueryMetrics {
+    /**
+     * Gets minimum execution time of query.
+     *
+     * @return Minimum execution time of query.
+     */
+    public long minimumTime();
+
+    /**
+     * Gets maximum execution time of query.
+     *
+     * @return Maximum execution time of query.
+     */
+    public long maximumTime();
+
+    /**
+     * Gets average execution time of query.
+     *
+     * @return Average execution time of query.
+     */
+    public double averageTime();
+
+    /**
+     * Gets total number execution of query.
+     *
+     * @return Number of executions.
+     */
+    public int executions();
+
+    /**
+     * Gets total number of times a query execution failed.
+     *
+     * @return Total number of times a query execution failed.
+     */
+    public int fails();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
index a01f4a9..376cd77 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryReducer.java
@@ -18,13 +18,24 @@
 package org.apache.ignite.cache.query;
 
 /**
- * TODO: Add class description.
- *
- * @author @java.author
- * @version @java.version
+ * Remote query result reducer. Collects data entries and can produce multiple
+ * results.
  */
 public interface QueryReducer<T, R> {
+    /**
+     * Collect next data entry.
+     *
+     * @param entry Data entry.
+     * @return {@code -1} If iteration over data set must be stopped and no result must be asked from the reducer,
+     *  {@code 0} if one of intermediate result is ready but iteration must continue if more data available,
+     *  {@code 1} if result is not ready and iteration must continue if more data available.
+     */
     public int collect(T entry);
 
+    /**
+     * Get reduce result (possibly intermediate).
+     *
+     * @return Result of reduction.
+     */
     public R reduce();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeMetadata.java
new file mode 100644
index 0000000..51c7736
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeMetadata.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+
+/**
+ * Cache query type metadata.
+ */
+public class QueryTypeMetadata {
+    /** Type name, e.g. class name. */
+    @GridToStringInclude
+    private String type;
+
+    /** Fields to be queried, in addition to indexed fields. */
+    @GridToStringInclude
+    private Map<String, Class<?>> qryFlds = new HashMap<>();
+
+    /** Fields to index in ascending order. */
+    @GridToStringInclude
+    private Map<String, Class<?>> ascFlds = new HashMap<>();
+
+    /** Fields to index in descending order. */
+    @GridToStringInclude
+    private Map<String, Class<?>> descFlds = new HashMap<>();
+
+    /** Fields to index as text. */
+    @GridToStringInclude
+    private Collection<String> txtFlds = new LinkedHashSet<>();
+
+    /** Fields to create group indexes for. */
+    @GridToStringInclude
+    private Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps;
+
+    /**
+     * Default constructor.
+     */
+    public QueryTypeMetadata() {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    public QueryTypeMetadata(QueryTypeMetadata src) {
+        type = src.getType();
+
+        qryFlds = new HashMap<>(src.getQueryFields());
+        ascFlds = new HashMap<>(src.getAscendingFields());
+        descFlds = new HashMap<>(src.getDescendingFields());
+        txtFlds = new HashSet<>(src.getTextFields());
+
+        grps = new HashMap<>(src.getGroups());
+    }
+
+    /**
+     * Gets type (e.g. class name).
+     *
+     * @return Type name.
+     */
+    public String getType() {
+        return type;
+    }
+
+    /**
+     * Sets type.
+     *
+     * @param cls Type class.
+     */
+    public void setType(Class<?> cls) {
+        setType(cls.getName());
+    }
+
+    /**
+     * Sets type.
+     *
+     * @param type Type name.
+     */
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    /**
+     * Gets query-enabled fields.
+     *
+     * @return Collection of fields available for query.
+     */
+    public Map<String, Class<?>> getQueryFields() {
+        return qryFlds;
+    }
+
+    /**
+     * Sets query fields map.
+     *
+     * @param qryFlds Query fields.
+     */
+    public void setQueryFields(Map<String, Class<?>> qryFlds) {
+        this.qryFlds = qryFlds;
+    }
+
+    /**
+     * Gets ascending-indexed fields.
+     *
+     * @return Map of ascending-indexed fields.
+     */
+    public Map<String, Class<?>> getAscendingFields() {
+        return ascFlds;
+    }
+
+    /**
+     * Sets ascending-indexed fields.
+     *
+     * @param ascFlds Map of ascending-indexed fields.
+     */
+    public void setAscendingFields(Map<String, Class<?>> ascFlds) {
+        this.ascFlds = ascFlds;
+    }
+
+    /**
+     * Gets descending-indexed fields.
+     *
+     * @return Map of descending-indexed fields.
+     */
+    public Map<String, Class<?>> getDescendingFields() {
+        return descFlds;
+    }
+
+    /**
+     * Sets descending-indexed fields.
+     *
+     * @param descFlds Map of descending-indexed fields.
+     */
+    public void setDescendingFields(Map<String, Class<?>> descFlds) {
+        this.descFlds = descFlds;
+    }
+
+    /**
+     * Gets text-indexed fields.
+     *
+     * @return Collection of text indexed fields.
+     */
+    public Collection<String> getTextFields() {
+        return txtFlds;
+    }
+
+    /**
+     * Sets text-indexed fields.
+     *
+     * @param txtFlds Text-indexed fields.
+     */
+    public void setTextFields(Collection<String> txtFlds) {
+        this.txtFlds = txtFlds;
+    }
+
+    /**
+     * Gets group-indexed fields.
+     *
+     * @return Map of group-indexed fields.
+     */
+    public Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> getGroups() {
+        return grps;
+    }
+
+    /**
+     * Sets group-indexed fields.
+     *
+     * @param grps Map of group-indexed fields from index name to index fields.
+     */
+    public void setGroups(Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps) {
+        this.grps = grps;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryTypeMetadata.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeResolver.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeResolver.java
new file mode 100644
index 0000000..f8bb017
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryTypeResolver.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+/**
+ * Interface allowing to override table name for portable objects stored in cache.
+ */
+public interface QueryTypeResolver {
+    /**
+     * Allows to override type name for portable objects being stored in cache.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @return Type name.
+     */
+    public String resolveTypeName(Object key, Object val);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
index a4a8dfc..9c756c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryGroupIndex.java
@@ -33,15 +33,6 @@ public @interface QueryGroupIndex {
     String name();
 
     /**
-     * If this index is unique.
-     *
-     * @return True if this index is unique, false otherwise.
-     * @deprecated No longer supported, will be ignored.
-     */
-    @Deprecated
-    boolean unique() default false;
-
-    /**
      * List of group indexes for type.
      */
     @SuppressWarnings("PublicInnerClass")

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
index c5f858b..0dfa7de 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
@@ -21,8 +21,8 @@ import java.lang.annotation.*;
 
 /**
  * Annotates fields for SQL queries. All fields that will be involved in SQL clauses must have
- * this annotation. For more information about cache queries see {@link org.apache.ignite.cache.query.CacheQuery} documentation.
- * @see org.apache.ignite.cache.query.CacheQuery
+ * this annotation. For more information about cache queries see {@link org.apache.ignite.internal.processors.cache.query.CacheQuery} documentation.
+ * @see org.apache.ignite.internal.processors.cache.query.CacheQuery
  */
 @Documented
 @Retention(RetentionPolicy.RUNTIME)
@@ -42,16 +42,6 @@ public @interface QuerySqlField {
     boolean index() default false;
 
     /**
-     * Specifies whether index should be unique or not. This property only
-     * makes sense if {@link #index()} property is set to {@code true}.
-     *
-     * @return {@code True} if field index should be unique.
-     * @deprecated No longer supported, will be ignored.
-     */
-    @Deprecated
-    boolean unique() default false;
-
-    /**
      * Specifies whether index should be in descending order or not. This property only
      * makes sense if {@link #index()} property is set to {@code true}.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
index 3015073..dbc6e22 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QueryTextField.java
@@ -22,8 +22,8 @@ import java.lang.annotation.*;
 /**
  * Annotation for fields or getters to be indexed for full text
  * search using {@code H2 TEXT} indexing. For more information
- * refer to {@link org.apache.ignite.cache.query.CacheQuery} documentation.
- * @see org.apache.ignite.cache.query.CacheQuery
+ * refer to {@link org.apache.ignite.internal.processors.cache.query.CacheQuery} documentation.
+ * @see org.apache.ignite.internal.processors.cache.query.CacheQuery
  */
 @Documented
 @Retention(RetentionPolicy.RUNTIME)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/configuration/IgniteQueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteQueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteQueryConfiguration.java
index d4a5e2f..0f2630d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteQueryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteQueryConfiguration.java
@@ -123,7 +123,7 @@ public class IgniteQueryConfiguration {
     }
 
     /**
-     * Sets classes with methods annotated by {@link org.apache.ignite.cache.query.CacheQuerySqlFunction}
+     * Sets classes with methods annotated by {@link org.apache.ignite.cache.query.QuerySqlFunction}
      * to be used as user-defined functions from SQL queries.
      *
      * @param idxCustomFuncClss List of classes.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryExecutedEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryExecutedEvent.java b/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryExecutedEvent.java
index c580e51..476a5fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryExecutedEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryExecutedEvent.java
@@ -19,6 +19,7 @@ package org.apache.ignite.events;
 
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryReadEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryReadEvent.java b/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryReadEvent.java
index dbf0b25..b872e85 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryReadEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/IgniteCacheQueryReadEvent.java
@@ -19,6 +19,7 @@ package org.apache.ignite.events;
 
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheWeakQueryIteratorsHolder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheWeakQueryIteratorsHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheWeakQueryIteratorsHolder.java
index 902cf12..df96a5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheWeakQueryIteratorsHolder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheWeakQueryIteratorsHolder.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jdk8.backport.*;
@@ -34,7 +34,7 @@ public class CacheWeakQueryIteratorsHolder<V> {
     private final ReferenceQueue<WeakQueryFutureIterator> refQueue = new ReferenceQueue<>();
 
     /** Iterators futures. */
-    private final Map<WeakReference<WeakQueryFutureIterator>, CacheQueryFuture<V>> futs =
+    private final Map<WeakReference<WeakQueryFutureIterator>,CacheQueryFuture<V>> futs =
         new ConcurrentHashMap8<>();
 
     /** Logger. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 3ed56c5..c6eaabb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
index 72b2505..fcf71a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index a5b73dc..52c83ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/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 33bc571..95da898 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
@@ -987,6 +987,12 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter<IgniteCach
     }
 
     /** {@inheritDoc} */
+    @Override public QueryMetrics queryMetrics() {
+        // TODO IGNITE-1.
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteCache<K, V> createAsyncInstance() {
         return new IgniteCacheProxy<>(ctx, delegate, prj, true);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
index 2bb679b..0b78b50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueries.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueries.java
new file mode 100644
index 0000000..ceac0f5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueries.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Facade for creating distributed queries. It contains various {@code 'createXxxQuery(..)'}
+ * methods for {@code SQL}, {@code TEXT}, and {@code SCAN} query creation (see {@link CacheQuery}
+ * for more information).
+ * <p>
+ * Instance of {@code CacheQueries} is obtained from cache projection as follows:
+ * <pre name="code" class="java">
+ * CacheQueries q = GridGain.grid().cache("myCache").queries();
+ * </pre>
+ */
+public interface CacheQueries<K, V> {
+    /**
+     * Creates user's SQL query, queried class, and query clause which is generally
+     * a where clause. For more information refer to {@link CacheQuery} documentation.
+     *
+     * @param cls Query class.
+     * @param clause Query clause.
+     * @return Created query.
+     */
+    public CacheQuery<Map.Entry<K, V>> createSqlQuery(Class<?> cls, String clause);
+
+    /**
+     * Creates user's SQL query, queried class, and query clause which is generally
+     * a where clause. For more information refer to {@link CacheQuery} documentation.
+     *
+     * @param clsName Query class name.
+     * @param clause Query clause.
+     * @return Created query.
+     */
+    public CacheQuery<Map.Entry<K, V>> createSqlQuery(String clsName, String clause);
+
+    /**
+     * Creates user's SQL fields query for given clause. For more information refer to
+     * {@link CacheQuery} documentation.
+     *
+     * @param qry Query.
+     * @return Created query.
+     */
+    public CacheQuery<List<?>> createSqlFieldsQuery(String qry);
+
+    /**
+     * Creates user's full text query, queried class, and query clause.
+     * For more information refer to {@link CacheQuery} documentation.
+     *
+     * @param clsName Query class name.
+     * @param search Search clause.
+     * @return Created query.
+     */
+    public CacheQuery<Map.Entry<K, V>> createFullTextQuery(String clsName, String search);
+
+    /**
+     * Creates user's full text query, queried class, and query clause.
+     * For more information refer to {@link CacheQuery} documentation.
+     *
+     * @param cls Query class.
+     * @param search Search clause.
+     * @return Created query.
+     */
+    public CacheQuery<Map.Entry<K, V>> createFullTextQuery(Class<?> cls, String search);
+
+    /**
+     * Creates user's predicate based scan query.
+     *
+     * @param filter Scan filter.
+     * @return Created query.
+     */
+    public CacheQuery<Map.Entry<K, V>> createScanQuery(@Nullable IgniteBiPredicate<K, V> filter);
+
+    /**
+     * Creates new continuous query.
+     * <p>
+     * For more information refer to {@link org.apache.ignite.cache.query.CacheContinuousQuery} documentation.
+     *
+     * @return Created continuous query.
+     * @see org.apache.ignite.cache.query.CacheContinuousQuery
+     */
+    public CacheContinuousQuery<K, V> createContinuousQuery();
+
+    /**
+     * Forces this cache to rebuild all search indexes of given value type. Sometimes indexes
+     * may hold references to objects that have already been removed from cache. Although
+     * not affecting query results, these objects may consume extra memory. Rebuilding
+     * indexes will remove any redundant references that may have temporarily got stuck
+     * inside in-memory index.
+     *
+     * @param cls Value type to rebuild indexes for.
+     *
+     * @return Future that will be completed when rebuilding of all indexes is finished.
+     */
+    public IgniteInternalFuture<?> rebuildIndexes(Class<?> cls);
+
+    /**
+     * Forces this cache to rebuild all search indexes of given value type. Sometimes indexes
+     * may hold references to objects that have already been removed from cache. Although
+     * not affecting query results, these objects may consume extra memory. Rebuilding
+     * indexes will remove any redundant references that may have temporarily got stuck
+     * inside in-memory index.
+     *
+     * @param typeName Value type name to rebuild indexes for.
+     *
+     * @return Future that will be completed when rebuilding of all indexes is finished.
+     */
+    public IgniteInternalFuture<?> rebuildIndexes(String typeName);
+
+    /**
+     * Forces this cache to rebuild search indexes of all types. Sometimes indexes
+     * may hold references to objects that have already been removed from cache. Although
+     * not affecting query results, these objects may consume extra memory. Rebuilding
+     * indexes will remove any redundant references that may have temporarily got stuck
+     * inside in-memory index.
+     *
+     * @return Future that will be completed when rebuilding of all indexes is finished.
+     */
+    public IgniteInternalFuture<?> rebuildAllIndexes();
+
+    /**
+     * Accumulated metrics for all queries executed for this cache.
+     *
+     * @return Cache query metrics.
+     */
+    public QueryMetrics metrics();
+
+    /**
+     * Resets accumulated metrics.
+     */
+    public void resetMetrics();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java
new file mode 100644
index 0000000..0468b93
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Main API for configuring and executing cache queries.
+ * <p>
+ * Cache queries are created from {@link CacheQueries} API via any of the available
+ * {@code createXXXQuery(...)} methods.
+ * <h1 class="header">SQL Queries</h1>
+ * {@code SQL} query allows to execute distributed cache
+ * queries using standard SQL syntax. All values participating in where clauses
+ * or joins must be annotated with {@link QuerySqlField} annotation. Query can be created
+ * with {@link CacheQueries#createSqlQuery(Class, String)} method.
+ * <h2 class="header">Field Queries</h2>
+ * By default {@code select} clause is ignored as query result contains full objects.
+ * If it is needed to select individual fields, use {@link CacheQueries#createSqlFieldsQuery(String)} method.
+ * This type of query replaces full objects with individual fields. Note that selected fields
+ * must be annotated with {@link QuerySqlField} annotation.
+ * <h2 class="header">Cross-Cache Queries</h2>
+ * You are allowed to query data from several caches. Cache that this query was created on is
+ * treated as default schema in this case. Other caches can be referenced by their names.
+ * <p>
+ * Note that cache name is case sensitive and has to always be specified in double quotes.
+ * Here is an example of cross cache query (note that 'replicated' and 'partitioned' are
+ * cache names for replicated and partitioned caches accordingly):
+ * <pre name="code" class="java">
+ * CacheQuery&lt;Map.Entry&lt;Integer, FactPurchase&gt;&gt; storePurchases = cache.queries().createSqlQuery(
+ *     Purchase.class,
+ *     "from \"replicated\".Store, \"partitioned\".Purchase where Store.id=Purchase.storeId and Store.id=?");
+ * </pre>
+ * <h2 class="header">Custom functions in SQL queries.</h2>
+ * It is possible to write custom Java methods and call then form SQL queries. These methods must be public static
+ * and annotated with {@link QuerySqlFunction}. Classes containing these methods must be registered in
+ * {@link org.apache.ignite.configuration.IgniteQueryConfiguration#setIndexCustomFunctionClasses(Class[])}.
+ * <h1 class="header">Full Text Queries</h1>
+ * GridGain supports full text queries based on Apache Lucene engine. This queries are created by
+ * {@link CacheQueries#createFullTextQuery(Class, String)} method. Note that all fields that
+ * are expected to show up in text query results must be annotated with {@link QueryTextField}
+ * annotation.
+ * <h1 class="header">Scan Queries</h1>
+ * Sometimes when it is known in advance that SQL query will cause a full data scan, or whenever data set
+ * is relatively small, the full scan query may be used. This query will iterate over all cache
+ * entries, skipping over entries that don't pass the optionally provided key-value filter
+ * (see {@link CacheQueries#createScanQuery(org.apache.ignite.lang.IgniteBiPredicate)} method).
+ * <h2 class="header">Limitations</h2>
+ * Data in GridGain cache is usually distributed across several nodes,
+ * so some queries may not work as expected. Keep in mind following limitations
+ * (not applied if data is queried from one node only):
+ * <ul>
+ *     <li>
+ *         {@code Group by} and {@code sort by} statements are applied separately
+ *         on each node, so result set will likely be incorrectly grouped or sorted
+ *         after results from multiple remote nodes are grouped together.
+ *     </li>
+ *     <li>
+ *         Aggregation functions like {@code sum}, {@code max}, {@code avg}, etc.
+ *         are also applied on each node. Therefore you will get several results
+ *         containing aggregated values, one for each node.
+ *     </li>
+ *     <li>
+ *         Joins will work correctly only if joined objects are stored in
+ *         collocated mode or at least one side of the join is stored in
+ *         {@link org.apache.ignite.cache.CacheMode#REPLICATED} cache. Refer to
+ *         {@link org.apache.ignite.cache.affinity.CacheAffinityKey} javadoc for more information about colocation.
+ *     </li>
+ * </ul>
+ * <h1 class="header">Query usage</h1>
+ * As an example, suppose we have data model consisting of {@code 'Employee'} and {@code 'Organization'}
+ * classes defined as follows:
+ * <pre name="code" class="java">
+ * public class Organization {
+ *     // Indexed field.
+ *     &#64;CacheQuerySqlField(index = true)
+ *     private long id;
+ *
+ *     // Indexed field.
+ *     &#64;CacheQuerySqlField(index = true)
+ *     private String name;
+ *     ...
+ * }
+ *
+ * public class Person {
+ *     // Indexed field.
+ *     &#64;CacheQuerySqlField(index = true)
+ *     private long id;
+ *
+ *     // Indexed field (Organization ID, used as a foreign key).
+ *     &#64;CacheQuerySqlField(index = true)
+ *     private long orgId;
+ *
+ *     // Without SQL field annotation, this field cannot be used in queries.
+ *     private String name;
+ *
+ *     // Not indexed field.
+ *     &#64;CacheQuerySqlField
+ *     private double salary;
+ *
+ *     // Index for text search.
+ *     &#64;CacheQueryTextField
+ *     private String resume;
+ *     ...
+ * }
+ * </pre>
+ * Then you can create and execute queries that check various salary ranges like so:
+ * <pre name="code" class="java">
+ * Cache&lt;Long, Person&gt; cache = G.grid().cache();
+ * ...
+ * // Create query which selects salaries based on range for all employees
+ * // that work for a certain company.
+ * CacheQuery&lt;Map.Entry&lt;Long, Person&gt;&gt; qry = cache.queries().createSqlQuery(Person.class,
+ *     "from Person, Organization where Person.orgId = Organization.id " +
+ *         "and Organization.name = ? and Person.salary &gt; ? and Person.salary &lt;= ?");
+ *
+ * // Query all nodes to find all cached GridGain employees
+ * // with salaries less than 1000.
+ * qry.execute("GridGain", 0, 1000);
+ *
+ * // Query only remote nodes to find all remotely cached GridGain employees
+ * // with salaries greater than 1000 and less than 2000.
+ * qry.projection(grid.remoteProjection()).execute("GridGain", 1000, 2000);
+ * </pre>
+ * Here is a possible query that will use Lucene text search to scan all resumes to
+ * check if employees have {@code Master} degree:
+ * <pre name="code" class="java">
+ * CacheQuery&lt;Map.Entry&lt;Long, Person&gt;&gt; mastersQry =
+ *     cache.queries().createFullTextQuery(Person.class, "Master");
+ *
+ * // Query all cache nodes.
+ * mastersQry.execute();
+ * </pre>
+ * <h1 class="header">Geo-Spatial Indexes and Queries</h1>
+ * GridGain also support <b>Geo-Spatial Indexes</b>. Here is an example of geo-spatial index:
+ * <pre name="code" class="java">
+ * private class MapPoint implements Serializable {
+ *     // Geospatial index.
+ *     &#64;CacheQuerySqlField(index = true)
+ *     private com.vividsolutions.jts.geom.Point location;
+ *
+ *     // Not indexed field.
+ *     &#64;CacheQuerySqlField
+ *     private String name;
+ *
+ *     public MapPoint(com.vividsolutions.jts.geom.Point location, String name) {
+ *         this.location = location;
+ *         this.name = name;
+ *     }
+ * }
+ * </pre>
+ * Example of spatial query on the geo-indexed field from above:
+ * <pre name="code" class="java">
+ * com.vividsolutions.jts.geom.GeometryFactory factory = new com.vividsolutions.jts.geom.GeometryFactory();
+ *
+ * com.vividsolutions.jts.geom.Polygon square = factory.createPolygon(new Coordinate[] {
+ *     new com.vividsolutions.jts.geom.Coordinate(0, 0),
+ *     new com.vividsolutions.jts.geom.Coordinate(0, 100),
+ *     new com.vividsolutions.jts.geom.Coordinate(100, 100),
+ *     new com.vividsolutions.jts.geom.Coordinate(100, 0),
+ *     new com.vividsolutions.jts.geom.Coordinate(0, 0)
+ * });
+ *
+ * Map.Entry<String, UserData> records = cache.queries().createSqlQuery(MapPoint.class, "select * from MapPoint where location && ?")
+ *     .queryArguments(square)
+ *     .execute()
+ *     .get();
+ * </pre>
+ */
+public interface CacheQuery<T> {
+    /** Default query page size. */
+    public static final int DFLT_PAGE_SIZE = 1024;
+
+    /**
+     * Sets result page size. If not provided, {@link #DFLT_PAGE_SIZE} will be used.
+     * Results are returned from queried nodes one page at a tme.
+     *
+     * @param pageSize Page size.
+     * @return {@code this} query instance for chaining.
+     */
+    public CacheQuery<T> pageSize(int pageSize);
+
+    /**
+     * Sets query timeout. {@code 0} means there is no timeout (this
+     * is a default value).
+     *
+     * @param timeout Query timeout.
+     * @return {@code this} query instance for chaining.
+     */
+    public CacheQuery<T> timeout(long timeout);
+
+    /**
+     * Sets whether or not to keep all query results local. If not - only the current page
+     * is kept locally. Default value is {@code true}.
+     *
+     * @param keepAll Keep results or not.
+     * @return {@code this} query instance for chaining.
+     */
+    public CacheQuery<T> keepAll(boolean keepAll);
+
+    /**
+     * Sets whether or not to include backup entries into query result. This flag
+     * is {@code false} by default.
+     *
+     * @param incBackups Query {@code includeBackups} flag.
+     * @return {@code this} query instance for chaining.
+     */
+    public CacheQuery<T> includeBackups(boolean incBackups);
+
+    /**
+     * Sets whether or not to deduplicate query result set. If this flag is {@code true}
+     * then query result will not contain some key more than once even if several nodes
+     * returned entries with the same keys. Default value is {@code false}.
+     *
+     * @param dedup Query {@code enableDedup} flag.
+     * @return {@code this} query instance for chaining.
+     */
+    public CacheQuery<T> enableDedup(boolean dedup);
+
+    /**
+     * Sets optional grid projection to execute this query on.
+     *
+     * @param prj Projection.
+     * @return {@code this} query instance for chaining.
+     */
+    public CacheQuery<T> projection(ClusterGroup prj);
+
+    /**
+     * Executes the query and returns the query future. Caller may decide to iterate
+     * over the returned future directly in which case the iterator may block until
+     * the next value will become available, or wait for the whole query to finish
+     * by calling any of the {@code 'get(..)'} methods on the returned future. If
+     * {@link #keepAll(boolean)} flag is set to {@code false}, then {@code 'get(..)'}
+     * methods will only return the last page received, otherwise all pages will be
+     * accumulated and returned to user as a collection.
+     * <p>
+     * Note that if the passed in grid projection is a local node, then query
+     * will be executed locally without distribution to other nodes.
+     * <p>
+     * Also note that query state cannot be changed (clause, timeout etc.), except
+     * arguments, if this method was called at least once.
+     *
+     * @param args Optional arguments.
+     * @return Future for the query result.
+     */
+    public CacheQueryFuture<T> execute(@Nullable Object... args);
+
+    /**
+     * Executes the query the same way as {@link #execute(Object...)} method but reduces result remotely.
+     *
+     * @param rmtReducer Remote reducer.
+     * @param args Optional arguments.
+     * @return Future for the query result.
+     */
+    public <R> CacheQueryFuture<R> execute(IgniteReducer<T, R> rmtReducer, @Nullable Object... args);
+
+    /**
+     * Executes the query the same way as {@link #execute(Object...)} method but transforms result remotely.
+     *
+     * @param rmtTransform Remote transformer.
+     * @param args Optional arguments.
+     * @return Future for the query result.
+     */
+    public <R> CacheQueryFuture<R> execute(IgniteClosure<T, R> rmtTransform, @Nullable Object... args);
+
+    /**
+     * Gets metrics for this query.
+     *
+     * @return Query metrics.
+     */
+    public QueryMetrics metrics();
+
+    /**
+     * Resets metrics for this query.
+     */
+    public void resetMetrics();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryFuture.java
new file mode 100644
index 0000000..b81f2e2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryFuture.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Cache query future returned by query execution.
+ * Refer to {@link CacheQuery} documentation for more information.
+ */
+public interface CacheQueryFuture<T> extends IgniteInternalFuture<Collection<T>> {
+    /**
+     * Returns number of elements that are already fetched and can
+     * be returned from {@link #next()} method without blocking.
+     *
+     * @return Number of fetched elements which are available immediately.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public int available() throws IgniteCheckedException;
+
+    /**
+     * Returns next element from result set.
+     * <p>
+     * This is a blocking call which will wait if there are no
+     * elements available immediately.
+     *
+     * @return Next fetched element or {@code null} if all the elements have been fetched.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable public T next() throws IgniteCheckedException;
+
+    /**
+     * Checks if all data is fetched by the query.
+     *
+     * @return {@code True} if all data is fetched, {@code false} otherwise.
+     */
+    @Override public boolean isDone();
+
+    /**
+     * Cancels this query future and stop receiving any further results for the query
+     * associated with this future.
+     *
+     * @return {@inheritDoc}
+     * @throws IgniteCheckedException {@inheritDoc}
+     */
+    @Override public boolean cancel() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
new file mode 100644
index 0000000..eab3e6d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+/**
+ * Cache query type.
+ * <p>
+ * Used in {@link org.apache.ignite.events.IgniteCacheQueryExecutedEvent} and {@link org.apache.ignite.events.IgniteCacheQueryReadEvent}
+ * to identify the type of query for which an event was fired.
+ *
+ * @see org.apache.ignite.events.IgniteCacheQueryExecutedEvent#queryType()
+ * @see org.apache.ignite.events.IgniteCacheQueryReadEvent#queryType()
+ */
+public enum CacheQueryType {
+    /** SQL query. */
+    SQL,
+
+    /** SQL fields query. */
+    SQL_FIELDS,
+
+    /** Full text query. */
+    FULL_TEXT,
+
+    /** Scan query. */
+    SCAN,
+
+    /** Continuous query. */
+    CONTINUOUS,
+
+    /** SPI query. */
+    SPI
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index cb7e9eb..3cd724c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
index 077e1c9..9e1af79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
index 4f39316..8d3c2be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.internal.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
index 1d215b2..4d6a63d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
@@ -33,7 +33,7 @@ import java.util.*;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.*;
 
 /**
- * {@link org.apache.ignite.cache.query.CacheQueries} implementation.
+ * {@link CacheQueries} implementation.
  */
 public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Externalizable {
     /** */
@@ -206,7 +206,7 @@ public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Ext
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryMetrics metrics() {
+    @Override public QueryMetrics metrics() {
         return ctx.queries().metrics();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
index 0266629..60b2e16 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
@@ -235,7 +235,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryMetrics metrics() {
+    @Override public QueryMetrics metrics() {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/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 d35d215..5ff160a 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
@@ -391,7 +391,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         return execute(null, rmtTransform, args);
     }
 
-    @Override public CacheQueryMetrics metrics() {
+    @Override public QueryMetrics metrics() {
         return metrics.copy();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
index dd385d1..f4fcb37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.future.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa0ee770/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index 4202c99..0c6b184 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.timeout.*;