You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/03/13 16:34:58 UTC

[06/50] [abbrv] ignite git commit: IGNITE-4814: Implemented correct registartion and unregistration of query types on cache start. Moved GridQueryProcessor statics to separate utility class. This closes #1612.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeCandidate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeCandidate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeCandidate.java
new file mode 100644
index 0000000..69cd3ca
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeCandidate.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.query;
+
+/**
+ * Type candidate which possibly will be registered.
+ */
+public class QueryTypeCandidate {
+    /** Type ID. */
+    private final QueryTypeIdKey typeId;
+
+    /** Alternative type ID. */
+    private final QueryTypeIdKey altTypeId;
+
+    /** Descriptor. */
+    private final QueryTypeDescriptorImpl desc;
+
+    /**
+     * Constructor.
+     *
+     * @param typeId Type ID.
+     * @param altTypeId Alternative type ID.
+     * @param desc Descriptor.
+     */
+    public QueryTypeCandidate(QueryTypeIdKey typeId, QueryTypeIdKey altTypeId, QueryTypeDescriptorImpl desc) {
+        this.typeId = typeId;
+        this.altTypeId = altTypeId;
+        this.desc = desc;
+    }
+
+    /**
+     * @return Type ID.
+     */
+    public QueryTypeIdKey typeId() {
+        return typeId;
+    }
+
+    /**
+     * @return Alternative type ID.
+     */
+    public QueryTypeIdKey alternativeTypeId() {
+        return altTypeId;
+    }
+
+    /**
+     * @return Descriptor.
+     */
+    public QueryTypeDescriptorImpl descriptor() {
+        return desc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
new file mode 100644
index 0000000..c033801
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -0,0 +1,989 @@
+/*
+ * 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.query;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.affinity.AffinityKeyMapper;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
+import org.apache.ignite.internal.processors.query.property.QueryClassProperty;
+import org.apache.ignite.internal.processors.query.property.QueryFieldAccessor;
+import org.apache.ignite.internal.processors.query.property.QueryMethodsAccessor;
+import org.apache.ignite.internal.processors.query.property.QueryPropertyAccessor;
+import org.apache.ignite.internal.processors.query.property.QueryReadOnlyMethodsAccessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
+
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Utility methods for queries.
+ */
+public class QueryUtils {
+    /** */
+    public static final String _VAL = "_val";
+
+    /** */
+    private static final Class<?> GEOMETRY_CLASS = U.classForName("com.vividsolutions.jts.geom.Geometry", null);
+
+    /** */
+    private static final Set<Class<?>> SQL_TYPES = new HashSet<>(F.<Class<?>>asList(
+        Integer.class,
+        Boolean.class,
+        Byte.class,
+        Short.class,
+        Long.class,
+        BigDecimal.class,
+        Double.class,
+        Float.class,
+        Time.class,
+        Timestamp.class,
+        java.util.Date.class,
+        java.sql.Date.class,
+        String.class,
+        UUID.class,
+        byte[].class
+    ));
+
+    /**
+     * Create type candidate for query entity.
+     *
+     * @param space Space.
+     * @param cctx Cache context.
+     * @param qryEntity Query entity.
+     * @param mustDeserializeClss Classes which must be deserialized.
+     * @return Type candidate.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static QueryTypeCandidate typeForQueryEntity(String space, GridCacheContext cctx, QueryEntity qryEntity,
+        List<Class<?>> mustDeserializeClss) throws IgniteCheckedException {
+        if (F.isEmpty(qryEntity.getValueType()))
+            throw new IgniteCheckedException("Value type is not set: " + qryEntity);
+
+        GridKernalContext ctx = cctx.kernalContext();
+        CacheConfiguration<?,?> ccfg = cctx.config();
+
+        boolean binaryEnabled = ctx.cacheObjects().isBinaryEnabled(ccfg);
+
+        CacheObjectContext coCtx = binaryEnabled ? ctx.cacheObjects().contextForCache(ccfg) : null;
+
+        QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl();
+
+        // Key and value classes still can be available if they are primitive or JDK part.
+        // We need that to set correct types for _key and _val columns.
+        Class<?> keyCls = U.classForName(qryEntity.getKeyType(), null);
+        Class<?> valCls = U.classForName(qryEntity.getValueType(), null);
+
+        // If local node has the classes and they are externalizable, we must use reflection properties.
+        boolean keyMustDeserialize = mustDeserializeBinary(ctx, keyCls);
+        boolean valMustDeserialize = mustDeserializeBinary(ctx, valCls);
+
+        boolean keyOrValMustDeserialize = keyMustDeserialize || valMustDeserialize;
+
+        if (keyCls == null)
+            keyCls = Object.class;
+
+        String simpleValType = ((valCls == null) ? typeName(qryEntity.getValueType()) : typeName(valCls));
+
+        desc.name(simpleValType);
+
+        desc.tableName(qryEntity.getTableName());
+
+        if (binaryEnabled && !keyOrValMustDeserialize) {
+            // Safe to check null.
+            if (SQL_TYPES.contains(valCls))
+                desc.valueClass(valCls);
+            else
+                desc.valueClass(Object.class);
+
+            if (SQL_TYPES.contains(keyCls))
+                desc.keyClass(keyCls);
+            else
+                desc.keyClass(Object.class);
+        }
+        else {
+            if (valCls == null)
+                throw new IgniteCheckedException("Failed to find value class in the node classpath " +
+                    "(use default marshaller to enable binary objects) : " + qryEntity.getValueType());
+
+            desc.valueClass(valCls);
+            desc.keyClass(keyCls);
+        }
+
+        desc.keyTypeName(qryEntity.getKeyType());
+        desc.valueTypeName(qryEntity.getValueType());
+
+        if (binaryEnabled && keyOrValMustDeserialize) {
+            if (keyMustDeserialize)
+                mustDeserializeClss.add(keyCls);
+
+            if (valMustDeserialize)
+                mustDeserializeClss.add(valCls);
+        }
+
+        QueryTypeIdKey typeId;
+        QueryTypeIdKey altTypeId = null;
+
+        if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
+            processBinaryMeta(ctx, qryEntity, desc);
+
+            typeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(qryEntity.getValueType()));
+
+            if (valCls != null)
+                altTypeId = new QueryTypeIdKey(space, valCls);
+
+            if (!cctx.customAffinityMapper() && qryEntity.getKeyType() != null) {
+                // Need to setup affinity key for distributed joins.
+                String affField = ctx.cacheObjects().affinityField(qryEntity.getKeyType());
+
+                if (affField != null)
+                    desc.affinityKey(affField);
+            }
+        }
+        else {
+            processClassMeta(qryEntity, desc, coCtx);
+
+            AffinityKeyMapper keyMapper = cctx.config().getAffinityMapper();
+
+            if (keyMapper instanceof GridCacheDefaultAffinityKeyMapper) {
+                String affField =
+                    ((GridCacheDefaultAffinityKeyMapper)keyMapper).affinityKeyPropertyName(desc.keyClass());
+
+                if (affField != null)
+                    desc.affinityKey(affField);
+            }
+
+            typeId = new QueryTypeIdKey(space, valCls);
+            altTypeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(qryEntity.getValueType()));
+        }
+
+        return new QueryTypeCandidate(typeId, altTypeId, desc);
+    }
+
+    /**
+     * Create type candidate for type metadata.
+     *
+     * @param space Space.
+     * @param cctx Cache context.
+     * @param meta Type metadata.
+     * @param mustDeserializeClss Classes which must be deserialized.
+     * @return Type candidate.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("deprecation")
+    @Nullable public static QueryTypeCandidate typeForCacheMetadata(String space, GridCacheContext cctx,
+        CacheTypeMetadata meta, List<Class<?>> mustDeserializeClss) throws IgniteCheckedException {
+        if (F.isEmpty(meta.getValueType()))
+            throw new IgniteCheckedException("Value type is not set: " + meta);
+
+        GridKernalContext ctx = cctx.kernalContext();
+        CacheConfiguration<?,?> ccfg = cctx.config();
+
+        boolean binaryEnabled = ctx.cacheObjects().isBinaryEnabled(ccfg);
+
+        CacheObjectContext coCtx = binaryEnabled ? ctx.cacheObjects().contextForCache(ccfg) : null;
+
+        if (meta.getQueryFields().isEmpty() && meta.getAscendingFields().isEmpty() &&
+            meta.getDescendingFields().isEmpty() && meta.getGroups().isEmpty())
+            return null;
+
+        QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl();
+
+        // Key and value classes still can be available if they are primitive or JDK part.
+        // We need that to set correct types for _key and _val columns.
+        Class<?> keyCls = U.classForName(meta.getKeyType(), null);
+        Class<?> valCls = U.classForName(meta.getValueType(), null);
+
+        // If local node has the classes and they are externalizable, we must use reflection properties.
+        boolean keyMustDeserialize = mustDeserializeBinary(ctx, keyCls);
+        boolean valMustDeserialize = mustDeserializeBinary(ctx, valCls);
+
+        boolean keyOrValMustDeserialize = keyMustDeserialize || valMustDeserialize;
+
+        if (keyCls == null)
+            keyCls = Object.class;
+
+        String simpleValType = meta.getSimpleValueType();
+
+        if (simpleValType == null)
+            simpleValType = typeName(meta.getValueType());
+
+        desc.name(simpleValType);
+
+        if (binaryEnabled && !keyOrValMustDeserialize) {
+            // Safe to check null.
+            if (SQL_TYPES.contains(valCls))
+                desc.valueClass(valCls);
+            else
+                desc.valueClass(Object.class);
+
+            if (SQL_TYPES.contains(keyCls))
+                desc.keyClass(keyCls);
+            else
+                desc.keyClass(Object.class);
+        }
+        else {
+            desc.valueClass(valCls);
+            desc.keyClass(keyCls);
+        }
+
+        desc.keyTypeName(meta.getKeyType());
+        desc.valueTypeName(meta.getValueType());
+
+        if (binaryEnabled && keyOrValMustDeserialize) {
+            if (keyMustDeserialize)
+                mustDeserializeClss.add(keyCls);
+
+            if (valMustDeserialize)
+                mustDeserializeClss.add(valCls);
+        }
+
+        QueryTypeIdKey typeId;
+        QueryTypeIdKey altTypeId = null;
+
+        if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
+            processBinaryMeta(ctx, meta, desc);
+
+            typeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(meta.getValueType()));
+
+            if (valCls != null)
+                altTypeId = new QueryTypeIdKey(space, valCls);
+        }
+        else {
+            processClassMeta(meta, desc, coCtx);
+
+            typeId = new QueryTypeIdKey(space, valCls);
+            altTypeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(meta.getValueType()));
+        }
+
+        return new QueryTypeCandidate(typeId, altTypeId, desc);
+    }
+    
+    /**
+     * Processes declarative metadata for class.
+     *
+     * @param meta Type metadata.
+     * @param d Type descriptor.
+     * @param coCtx Cache object context.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("deprecation")
+    private static void processClassMeta(CacheTypeMetadata meta, QueryTypeDescriptorImpl d, CacheObjectContext coCtx)
+        throws IgniteCheckedException {
+        Map<String,String> aliases = meta.getAliases();
+
+        if (aliases == null)
+            aliases = Collections.emptyMap();
+
+        Class<?> keyCls = d.keyClass();
+        Class<?> valCls = d.valueClass();
+
+        assert keyCls != null;
+        assert valCls != null;
+
+        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet())
+            addToIndex(d, keyCls, valCls, entry.getKey(), entry.getValue(), 0, IndexType.ASC, null, aliases, coCtx);
+
+        for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet())
+            addToIndex(d, keyCls, valCls, entry.getKey(), entry.getValue(), 0, IndexType.DESC, null, aliases, coCtx);
+
+        for (String txtField : meta.getTextFields())
+            addToIndex(d, keyCls, valCls, txtField, String.class, 0, IndexType.TEXT, null, aliases, coCtx);
+
+        Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps = meta.getGroups();
+
+        if (grps != null) {
+            for (Map.Entry<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> entry : grps.entrySet()) {
+                String idxName = entry.getKey();
+
+                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
+
+                int order = 0;
+
+                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
+                    Boolean descending = idxField.getValue().get2();
+
+                    if (descending == null)
+                        descending = false;
+
+                    addToIndex(d, keyCls, valCls, idxField.getKey(), idxField.getValue().get1(), order,
+                        descending ? IndexType.DESC : IndexType.ASC, idxName, aliases, coCtx);
+
+                    order++;
+                }
+            }
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
+            QueryClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                entry.getKey(),
+                entry.getValue(),
+                aliases,
+                coCtx);
+
+            d.addProperty(prop, false);
+        }
+    }
+    
+    /**
+     * @param d Type descriptor.
+     * @param keyCls Key class.
+     * @param valCls Value class.
+     * @param pathStr Path string.
+     * @param resType Result type.
+     * @param idxOrder Order number in index or {@code -1} if no need to index.
+     * @param idxType Index type.
+     * @param idxName Index name.
+     * @param aliases Aliases.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static void addToIndex(
+        QueryTypeDescriptorImpl d,
+        Class<?> keyCls,
+        Class<?> valCls,
+        String pathStr,
+        Class<?> resType,
+        int idxOrder,
+        IndexType idxType,
+        String idxName,
+        Map<String,String> aliases,
+        CacheObjectContext coCtx
+    ) throws IgniteCheckedException {
+        String propName;
+        Class<?> propCls;
+
+        if (_VAL.equals(pathStr)) {
+            propName = _VAL;
+            propCls = valCls;
+        }
+        else {
+            QueryClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                pathStr,
+                resType,
+                aliases,
+                coCtx);
+
+            d.addProperty(prop, false);
+
+            propName = prop.name();
+            propCls = prop.type();
+        }
+
+        if (idxType != null) {
+            if (idxName == null)
+                idxName = propName + "_idx";
+
+            if (idxOrder == 0) // Add index only on the first field.
+                d.addIndex(idxName, isGeometryClass(propCls) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
+
+            if (idxType == IndexType.TEXT)
+                d.addFieldToTextIndex(propName);
+            else
+                d.addFieldToIndex(idxName, propName, idxOrder, idxType == IndexType.DESC);
+        }
+    }
+    
+    /**
+     * Processes declarative metadata for binary object.
+     *
+     * @param ctx Kernal context.
+     * @param meta Declared metadata.
+     * @param d Type descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("deprecation")
+    public static void processBinaryMeta(GridKernalContext ctx, CacheTypeMetadata meta, QueryTypeDescriptorImpl d)
+        throws IgniteCheckedException {
+        Map<String,String> aliases = meta.getAliases();
+
+        if (aliases == null)
+            aliases = Collections.emptyMap();
+
+        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
+            QueryBinaryProperty prop = buildBinaryProperty(ctx, entry.getKey(), entry.getValue(), aliases, null);
+
+            d.addProperty(prop, false);
+
+            String idxName = prop.name() + "_idx";
+
+            d.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
+
+            d.addFieldToIndex(idxName, prop.name(), 0, false);
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
+            QueryBinaryProperty prop = buildBinaryProperty(ctx, entry.getKey(), entry.getValue(), aliases, null);
+
+            d.addProperty(prop, false);
+
+            String idxName = prop.name() + "_idx";
+
+            d.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
+
+            d.addFieldToIndex(idxName, prop.name(), 0, true);
+        }
+
+        for (String txtIdx : meta.getTextFields()) {
+            QueryBinaryProperty prop = buildBinaryProperty(ctx, txtIdx, String.class, aliases, null);
+
+            d.addProperty(prop, false);
+
+            d.addFieldToTextIndex(prop.name());
+        }
+
+        Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps = meta.getGroups();
+
+        if (grps != null) {
+            for (Map.Entry<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> entry : grps.entrySet()) {
+                String idxName = entry.getKey();
+
+                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
+
+                int order = 0;
+
+                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
+                    QueryBinaryProperty prop = buildBinaryProperty(ctx, idxField.getKey(), idxField.getValue().get1(),
+                        aliases, null);
+
+                    d.addProperty(prop, false);
+
+                    Boolean descending = idxField.getValue().get2();
+
+                    d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
+
+                    order++;
+                }
+            }
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
+            QueryBinaryProperty prop = buildBinaryProperty(ctx, entry.getKey(), entry.getValue(), aliases, null);
+
+            if (!d.properties().containsKey(prop.name()))
+                d.addProperty(prop, false);
+        }
+    }
+    
+    /**
+     * Processes declarative metadata for binary object.
+     *
+     * @param ctx Kernal context.
+     * @param qryEntity Declared metadata.
+     * @param d Type descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static void processBinaryMeta(GridKernalContext ctx, QueryEntity qryEntity, QueryTypeDescriptorImpl d)
+        throws IgniteCheckedException {
+        Map<String,String> aliases = qryEntity.getAliases();
+
+        if (aliases == null)
+            aliases = Collections.emptyMap();
+
+        Set<String> keyFields = qryEntity.getKeyFields();
+
+        // We have to distinguish between empty and null keyFields when the key is not of SQL type -
+        // when a key is not of SQL type, absence of a field in nonnull keyFields tell us that this field
+        // is a value field, and null keyFields tells us that current configuration
+        // does not tell us anything about this field's ownership.
+        boolean hasKeyFields = (keyFields != null);
+
+        boolean isKeyClsSqlType = isSqlType(d.keyClass());
+
+        if (hasKeyFields && !isKeyClsSqlType) {
+            //ensure that 'keyFields' is case sensitive subset of 'fields'
+            for (String keyField : keyFields) {
+                if (!qryEntity.getFields().containsKey(keyField))
+                    throw new IgniteCheckedException("QueryEntity 'keyFields' property must be a subset of keys " +
+                        "from 'fields' property (case sensitive): " + keyField);
+            }
+        }
+
+        for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
+            Boolean isKeyField;
+
+            if (isKeyClsSqlType) // We don't care about keyFields in this case - it might be null, or empty, or anything
+                isKeyField = false;
+            else
+                isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
+
+            QueryBinaryProperty prop = buildBinaryProperty(ctx, entry.getKey(),
+                U.classForName(entry.getValue(), Object.class, true), aliases, isKeyField);
+
+            d.addProperty(prop, false);
+        }
+
+        processIndexes(qryEntity, d);
+    }
+    
+    /**
+     * Processes declarative metadata for binary object.
+     *
+     * @param qryEntity Declared metadata.
+     * @param d Type descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static void processClassMeta(QueryEntity qryEntity, QueryTypeDescriptorImpl d, CacheObjectContext coCtx)
+        throws IgniteCheckedException {
+        Map<String,String> aliases = qryEntity.getAliases();
+
+        if (aliases == null)
+            aliases = Collections.emptyMap();
+
+        for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
+            QueryClassProperty prop = buildClassProperty(
+                d.keyClass(),
+                d.valueClass(),
+                entry.getKey(),
+                U.classForName(entry.getValue(), Object.class),
+                aliases,
+                coCtx);
+
+            d.addProperty(prop, false);
+        }
+
+        processIndexes(qryEntity, d);
+    }
+    
+    /**
+     * Processes indexes based on query entity.
+     *
+     * @param qryEntity Query entity to process.
+     * @param d Type descriptor to populate.
+     * @throws IgniteCheckedException If failed to build index information.
+     */
+    private static void processIndexes(QueryEntity qryEntity, QueryTypeDescriptorImpl d) throws IgniteCheckedException {
+        if (!F.isEmpty(qryEntity.getIndexes())) {
+            Map<String, String> aliases = qryEntity.getAliases();
+
+            if (aliases == null)
+                aliases = Collections.emptyMap();
+
+            for (QueryIndex idx : qryEntity.getIndexes()) {
+                String idxName = idx.getName();
+
+                if (idxName == null)
+                    idxName = QueryEntity.defaultIndexName(idx);
+
+                QueryIndexType idxTyp = idx.getIndexType();
+
+                if (idxTyp == QueryIndexType.SORTED || idxTyp == QueryIndexType.GEOSPATIAL) {
+                    d.addIndex(idxName, idxTyp);
+
+                    int i = 0;
+
+                    for (Map.Entry<String, Boolean> entry : idx.getFields().entrySet()) {
+                        String field = entry.getKey();
+                        boolean asc = entry.getValue();
+
+                        String alias = aliases.get(field);
+
+                        if (alias != null)
+                            field = alias;
+
+                        d.addFieldToIndex(idxName, field, i++, !asc);
+                    }
+                }
+                else if (idxTyp == QueryIndexType.FULLTEXT){
+                    for (String field : idx.getFields().keySet()) {
+                        String alias = aliases.get(field);
+
+                        if (alias != null)
+                            field = alias;
+
+                        d.addFieldToTextIndex(field);
+                    }
+                }
+                else if (idxTyp != null)
+                    throw new IllegalArgumentException("Unsupported index type [idx=" + idx.getName() +
+                        ", typ=" + idxTyp + ']');
+                else
+                    throw new IllegalArgumentException("Index type is not set: " + idx.getName());
+            }
+        }
+    }
+    
+    /**
+     * Builds binary object property.
+     *
+     * @param ctx Kernal context.
+     * @param pathStr String representing path to the property. May contains dots '.' to identify
+     *      nested fields.
+     * @param resType Result type.
+     * @param aliases Aliases.
+     * @param isKeyField Key ownership flag, as defined in {@link QueryEntity#keyFields}: {@code true} if field belongs
+     *      to key, {@code false} if it belongs to value, {@code null} if QueryEntity#keyFields is null.
+     * @return Binary property.
+     */
+    public static QueryBinaryProperty buildBinaryProperty(GridKernalContext ctx, String pathStr, Class<?> resType,
+        Map<String, String> aliases, @Nullable Boolean isKeyField) throws IgniteCheckedException {
+        String[] path = pathStr.split("\\.");
+
+        QueryBinaryProperty res = null;
+
+        StringBuilder fullName = new StringBuilder();
+
+        for (String prop : path) {
+            if (fullName.length() != 0)
+                fullName.append('.');
+
+            fullName.append(prop);
+
+            String alias = aliases.get(fullName.toString());
+
+            // The key flag that we've found out is valid for the whole path.
+            res = new QueryBinaryProperty(ctx, prop, res, resType, isKeyField, alias);
+        }
+
+        return res;
+    }
+    
+    /**
+     * @param keyCls Key class.
+     * @param valCls Value class.
+     * @param pathStr Path string.
+     * @param resType Result type.
+     * @param aliases Aliases.
+     * @return Class property.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static QueryClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr,
+        Class<?> resType, Map<String,String> aliases, CacheObjectContext coCtx) throws IgniteCheckedException {
+        QueryClassProperty res = buildClassProperty(
+            true,
+            keyCls,
+            pathStr,
+            resType,
+            aliases,
+            coCtx);
+
+        if (res == null) // We check key before value consistently with BinaryProperty.
+            res = buildClassProperty(false, valCls, pathStr, resType, aliases, coCtx);
+
+        if (res == null)
+            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' of type '" +
+                resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
+                "Make sure that one of these classes contains respective getter method or field.");
+
+        return res;
+    }
+    
+    /**
+     * @param key If this is a key property.
+     * @param cls Source type class.
+     * @param pathStr String representing path to the property. May contains dots '.' to identify nested fields.
+     * @param resType Expected result type.
+     * @param aliases Aliases.
+     * @return Property instance corresponding to the given path.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public static QueryClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
+        Map<String,String> aliases, CacheObjectContext coCtx) {
+        String[] path = pathStr.split("\\.");
+
+        QueryClassProperty res = null;
+
+        StringBuilder fullName = new StringBuilder();
+
+        for (String prop : path) {
+            if (fullName.length() != 0)
+                fullName.append('.');
+
+            fullName.append(prop);
+
+            String alias = aliases.get(fullName.toString());
+
+            QueryPropertyAccessor accessor = findProperty(prop, cls);
+
+            if (accessor == null)
+                return null;
+
+            QueryClassProperty tmp = new QueryClassProperty(accessor, key, alias, coCtx);
+
+            tmp.parent(res);
+
+            cls = tmp.type();
+
+            res = tmp;
+        }
+
+        if (!U.box(resType).isAssignableFrom(U.box(res.type())))
+            return null;
+
+        return res;
+    }
+    
+    /**
+     * Find a member (either a getter method or a field) with given name of given class.
+     * @param prop Property name.
+     * @param cls Class to search for a member in.
+     * @return Member for given name.
+     */
+    @Nullable private static QueryPropertyAccessor findProperty(String prop, Class<?> cls) {
+        StringBuilder getBldr = new StringBuilder("get");
+        getBldr.append(prop);
+        getBldr.setCharAt(3, Character.toUpperCase(getBldr.charAt(3)));
+
+        StringBuilder setBldr = new StringBuilder("set");
+        setBldr.append(prop);
+        setBldr.setCharAt(3, Character.toUpperCase(setBldr.charAt(3)));
+
+        try {
+            Method getter = cls.getMethod(getBldr.toString());
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name like 'setXxx' and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(setBldr.toString(), getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new QueryReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new QueryMethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignore) {
+            // No-op.
+        }
+
+        getBldr = new StringBuilder("is");
+        getBldr.append(prop);
+        getBldr.setCharAt(2, Character.toUpperCase(getBldr.charAt(2)));
+
+        // We do nothing about setBldr here as it corresponds to setProperty name which is what we need
+        // for boolean property setter as well
+        try {
+            Method getter = cls.getMethod(getBldr.toString());
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name like 'setXxx' and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(setBldr.toString(), getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new QueryReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new QueryMethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignore) {
+            // No-op.
+        }
+
+        Class cls0 = cls;
+
+        while (cls0 != null)
+            try {
+                return new QueryFieldAccessor(cls0.getDeclaredField(prop));
+            }
+            catch (NoSuchFieldException ignored) {
+                cls0 = cls0.getSuperclass();
+            }
+
+        try {
+            Method getter = cls.getMethod(prop);
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(prop, getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new QueryReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new QueryMethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignored) {
+            // No-op.
+        }
+
+        // No luck.
+        return null;
+    }
+
+    /**
+     * Check whether type still must be deserialized when binary marshaller is set.
+     *
+     * @param ctx Kernal context.
+     * @param cls Class.
+     * @return {@code True} if will be deserialized.
+     */
+    private static boolean mustDeserializeBinary(GridKernalContext ctx, Class cls) {
+        if (cls != null && ctx.config().getMarshaller() instanceof BinaryMarshaller) {
+            CacheObjectBinaryProcessorImpl proc0 = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects();
+
+            return proc0.binaryContext().mustDeserialize(cls);
+        }
+        else
+            return false;
+    }
+
+    /**
+     * Checks if the given class can be mapped to a simple SQL type.
+     *
+     * @param cls Class.
+     * @return {@code true} If can.
+     */
+    public static boolean isSqlType(Class<?> cls) {
+        cls = U.box(cls);
+
+        return SQL_TYPES.contains(cls) || QueryUtils.isGeometryClass(cls);
+    }
+
+    /**
+     * Checks if the given class is GEOMETRY.
+     *
+     * @param cls Class.
+     * @return {@code true} If this is geometry.
+     */
+    public static boolean isGeometryClass(Class<?> cls) {
+        return GEOMETRY_CLASS != null && GEOMETRY_CLASS.isAssignableFrom(cls);
+    }
+
+    /**
+     * Gets type name by class.
+     *
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    public static String typeName(String clsName) {
+        int pkgEnd = clsName.lastIndexOf('.');
+
+        if (pkgEnd >= 0 && pkgEnd < clsName.length() - 1)
+            clsName = clsName.substring(pkgEnd + 1);
+
+        if (clsName.endsWith("[]"))
+            clsName = clsName.substring(0, clsName.length() - 2) + "_array";
+
+        int parentEnd = clsName.lastIndexOf('$');
+
+        if (parentEnd >= 0)
+            clsName = clsName.substring(parentEnd + 1);
+
+        return clsName;
+    }
+
+    /**
+     * Gets type name by class.
+     *
+     * @param cls Class.
+     * @return Type name.
+     */
+    public static String typeName(Class<?> cls) {
+        String typeName = cls.getSimpleName();
+
+        // To protect from failure on anonymous classes.
+        if (F.isEmpty(typeName)) {
+            String pkg = cls.getPackage().getName();
+
+            typeName = cls.getName().substring(pkg.length() + (pkg.isEmpty() ? 0 : 1));
+        }
+
+        if (cls.isArray()) {
+            assert typeName.endsWith("[]");
+
+            typeName = typeName.substring(0, typeName.length() - 2) + "_array";
+        }
+
+        return typeName;
+    }
+
+    /**
+     * @param timeout Timeout.
+     * @param timeUnit Time unit.
+     * @return Converted time.
+     */
+    public static int validateTimeout(int timeout, TimeUnit timeUnit) {
+        A.ensure(timeUnit != TimeUnit.MICROSECONDS && timeUnit != TimeUnit.NANOSECONDS,
+            "timeUnit minimal resolution is millisecond.");
+
+        A.ensure(timeout >= 0, "timeout value should be non-negative.");
+
+        long tmp = TimeUnit.MILLISECONDS.convert(timeout, timeUnit);
+
+        return (int) tmp;
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @return {@code true} If query index must be enabled for this cache.
+     */
+    public static boolean isEnabled(CacheConfiguration<?,?> ccfg) {
+        return !F.isEmpty(ccfg.getIndexedTypes()) ||
+            !F.isEmpty(ccfg.getTypeMetadata()) ||
+            !F.isEmpty(ccfg.getQueryEntities());
+    }
+
+    /**
+     * Private constructor.
+     */
+    private QueryUtils() {
+        // No-op.
+    }
+    /**
+     * The way to index.
+     */
+    private enum IndexType {
+        /** Ascending index. */
+        ASC,
+
+        /** Descending index. */
+        DESC,
+
+        /** Text index. */
+        TEXT
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
index ef914c9..dba16e2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
@@ -69,19 +69,19 @@ public class QueryBinaryProperty implements GridQueryProperty {
      * Constructor.
      *
      * @param ctx Kernal context.
-     * @param log Logger.
      * @param propName Property name.
      * @param parent Parent property.
      * @param type Result type.
      * @param key {@code true} if key property, {@code false} otherwise, {@code null}  if unknown.
      * @param alias Field alias.
      */
-    public QueryBinaryProperty(GridKernalContext ctx, IgniteLogger log, String propName, QueryBinaryProperty parent,
+    public QueryBinaryProperty(GridKernalContext ctx, String propName, QueryBinaryProperty parent,
         Class<?> type, @Nullable Boolean key, String alias) {
         super();
 
         this.ctx = ctx;
-        this.log = log;
+
+        log = ctx.log(QueryBinaryProperty.class);
 
         this.propName = propName;
         this.alias = F.isEmpty(alias) ? propName : alias;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index a502479..cbf39f6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -55,10 +55,10 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
@@ -920,7 +920,7 @@ public class DmlStatementsProcessor {
 
         Object key = plan.keySupplier.apply(row);
 
-        if (GridQueryProcessor.isSqlType(desc.keyClass())) {
+        if (QueryUtils.isSqlType(desc.keyClass())) {
             assert plan.keyColIdx != -1;
 
             key = convert(key, rowDesc, desc.keyClass(), plan.colTypes[plan.keyColIdx]);
@@ -928,7 +928,7 @@ public class DmlStatementsProcessor {
 
         Object val = plan.valSupplier.apply(row);
 
-        if (GridQueryProcessor.isSqlType(desc.valueClass())) {
+        if (QueryUtils.isSqlType(desc.valueClass())) {
             assert plan.valColIdx != -1;
 
             val = convert(val, rowDesc, desc.valueClass(), plan.colTypes[plan.valColIdx]);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index a4e49e9..4438343 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -2202,14 +2202,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void registerCache(GridCacheContext<?, ?> cctx, CacheConfiguration<?,?> ccfg)
+    @Override public void registerCache(String spaceName, GridCacheContext<?, ?> cctx, CacheConfiguration<?,?> ccfg)
         throws IgniteCheckedException {
         String schema = schemaNameFromCacheConf(ccfg);
 
-        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), schema, cctx, ccfg)) != null)
-            throw new IgniteCheckedException("Cache already registered: " + U.maskName(ccfg.getName()));
+        if (schemas.putIfAbsent(schema, new Schema(spaceName, schema, cctx, ccfg)) != null)
+            throw new IgniteCheckedException("Cache already registered: " + U.maskName(spaceName));
 
-        space2schema.put(emptyIfNull(ccfg.getName()), schema);
+        space2schema.put(emptyIfNull(spaceName), schema);
 
         createSchema(schema);
 
@@ -2217,13 +2217,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void unregisterCache(CacheConfiguration<?, ?> ccfg) {
-        String schema = schema(ccfg.getName());
+    @Override public void unregisterCache(String spaceName) {
+        String schema = schema(spaceName);
         Schema rmv = schemas.remove(schema);
 
         if (rmv != null) {
             space2schema.remove(emptyIfNull(rmv.spaceName));
-            mapQryExec.onCacheStop(ccfg.getName());
+            mapQryExec.onCacheStop(spaceName);
 
             rmv.onDrop();
 
@@ -2231,14 +2231,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 dropSchema(schema);
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(ccfg.getName()), e);
+                U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(spaceName), e);
             }
 
             for (Iterator<Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
                 it.hasNext();) {
                 Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery> e = it.next();
 
-                if (F.eq(e.getKey().space, ccfg.getName()))
+                if (F.eq(e.getKey().space, spaceName))
                     it.remove();
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 6a373d0..77946dd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -26,10 +26,10 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
@@ -319,7 +319,7 @@ public final class UpdatePlanBuilder {
         final Class<?> cls = key ? U.firstNotNull(U.classForName(desc.keyTypeName(), null), desc.keyClass())
             : desc.valueClass();
 
-        boolean isSqlType = GridQueryProcessor.isSqlType(cls);
+        boolean isSqlType = QueryUtils.isSqlType(cls);
 
         // If we don't need to construct anything from scratch, just return value from given list.
         if (isSqlType || !hasProps) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryLoadSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryLoadSelfTest.java
index ca612a8..1933226 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryLoadSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryLoadSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -110,7 +110,7 @@ public class IgniteCacheQueryLoadSelfTest extends GridCommonAbstractTest {
      */
     private long size(Class<?> cls) throws IgniteCheckedException {
         return (Long)grid().cache(null).query(
-            new SqlFieldsQuery("select count(*) from " + GridQueryProcessor.typeName(cls)).setLocal(true))
+            new SqlFieldsQuery("select count(*) from " + QueryUtils.typeName(cls)).setLocal(true))
             .getAll().get(0).get(0);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d6ff06/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 56ff221..2d99010 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -102,8 +102,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
     protected void startIndexing(IgniteH2Indexing spi) throws Exception {
         spi.start(null, null);
 
-        spi.registerCache(null, cacheCfg("A"));
-        spi.registerCache(null, cacheCfg("B"));
+        CacheConfiguration ccfgA = cacheCfg("A");
+        CacheConfiguration ccfgB = cacheCfg("B");
+
+        spi.registerCache(ccfgA.getName(), null, ccfgA);
+        spi.registerCache(ccfgB.getName(), null, ccfgB);
     }
 
     /**