You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/03/24 14:12:31 UTC

[41/50] [abbrv] ignite git commit: Merge branch master ignite-2.0 to ignite-3477

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 0000000,20570ab..b96b9f8
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,1003 +1,1003 @@@
+ /*
+  * 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);
++                d.addIndex(idxName, isGeometryClass(propCls) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED, 0);
+ 
+             if (idxType == IndexType.TEXT)
+                 d.addFieldToTextIndex(propName);
+             else
 -                d.addFieldToIndex(idxName, propName, idxOrder, idxType == IndexType.DESC);
++                d.addFieldToIndex(idxName, propName, idxOrder, 0, 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.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED, 0);
+ 
 -            d.addFieldToIndex(idxName, prop.name(), 0, false);
++            d.addFieldToIndex(idxName, prop.name(), 0, 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.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED, 0);
+ 
 -            d.addFieldToIndex(idxName, prop.name(), 0, true);
++            d.addFieldToIndex(idxName, prop.name(), 0, 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);
++                    d.addFieldToIndex(idxName, prop.name(), order, 0, 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);
++                    d.addIndex(idxName, idxTyp, idx.getInlineSize());
+ 
+                     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);
++                        d.addFieldToIndex(idxName, field, i++, idx.getInlineSize(), !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(propertyInitializationExceptionMessage(keyCls, valCls, pathStr, resType));
+ 
+         return res;
+     }
+ 
+     /**
+      * Exception message to compare in tests.
+      *
+      * @param keyCls key class
+      * @param valCls value class
+      * @param pathStr property name
+      * @param resType property type
+      * @return Exception message.
+      */
+     public static String propertyInitializationExceptionMessage(Class<?> keyCls, Class<?> valCls, String pathStr,
+         Class<?> resType) {
+         return "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.";
+     }
+     
+     /**
+      * @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/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
index 0000000,dba16e2..3b42a78
mode 000000,100644..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
@@@ -1,0 -1,267 +1,269 @@@
+ /*
+  * 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.property;
+ 
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.IgniteLogger;
+ import org.apache.ignite.binary.BinaryField;
+ import org.apache.ignite.binary.BinaryObject;
+ import org.apache.ignite.binary.BinaryObjectBuilder;
+ import org.apache.ignite.binary.BinaryType;
+ import org.apache.ignite.internal.GridKernalContext;
+ import org.apache.ignite.internal.binary.BinaryObjectEx;
+ import org.apache.ignite.internal.binary.BinaryObjectExImpl;
+ import org.apache.ignite.internal.processors.query.GridQueryProperty;
+ import org.apache.ignite.internal.util.typedef.F;
+ import org.apache.ignite.internal.util.typedef.internal.U;
+ import org.jetbrains.annotations.Nullable;
+ 
+ /**
+  * Binary property.
+  */
+ public class QueryBinaryProperty implements GridQueryProperty {
+     /** Kernal context. */
+     private final GridKernalContext ctx;
+ 
+     /** Logger. */
+     private final IgniteLogger log;
+ 
+     /** Property name. */
+     private String propName;
+ 
+     /** */
+     private String alias;
+ 
+     /** Parent property. */
+     private QueryBinaryProperty parent;
+ 
+     /** Result class. */
+     private Class<?> type;
+ 
+     /** */
+     private volatile int isKeyProp;
+ 
+     /** Binary field to speed-up deserialization. */
+     private volatile BinaryField field;
+ 
+     /** Flag indicating that we already tried to take a field. */
+     private volatile boolean fieldTaken;
+ 
+     /** Whether user was warned about missing property. */
+     private volatile boolean warned;
+ 
+     /**
+      * Constructor.
+      *
+      * @param ctx Kernal context.
+      * @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, String propName, QueryBinaryProperty parent,
+         Class<?> type, @Nullable Boolean key, String alias) {
 -        super();
+ 
+         this.ctx = ctx;
+ 
+         log = ctx.log(QueryBinaryProperty.class);
+ 
+         this.propName = propName;
+         this.alias = F.isEmpty(alias) ? propName : alias;
+         this.parent = parent;
+         this.type = type;
+ 
+         if (key != null)
+             this.isKeyProp = key ? 1 : -1;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Object value(Object key, Object val) throws IgniteCheckedException {
+         Object obj;
+ 
+         if (parent != null) {
+             obj = parent.value(key, val);
+ 
+             if (obj == null)
+                 return null;
+ 
+             if (!ctx.cacheObjects().isBinaryObject(obj))
+                 throw new IgniteCheckedException("Non-binary object received as a result of property extraction " +
+                     "[parent=" + parent + ", propName=" + propName + ", obj=" + obj + ']');
+         }
+         else {
+             int isKeyProp0 = isKeyProp;
+ 
+             if (isKeyProp0 == 0) {
+                 // Key is allowed to be a non-binary object here.
+                 // We check key before value consistently with ClassProperty.
+                 if (key instanceof BinaryObject && ((BinaryObject)key).hasField(propName))
+                     isKeyProp = isKeyProp0 = 1;
+                 else if (val instanceof BinaryObject && ((BinaryObject)val).hasField(propName))
+                     isKeyProp = isKeyProp0 = -1;
+                 else {
+                     if (!warned) {
+                         U.warn(log, "Neither key nor value have property \"" + propName + "\" " +
+                             "(is cache indexing configured correctly?)");
+ 
+                         warned = true;
+                     }
+ 
+                     return null;
+                 }
+             }
+ 
+             obj = isKeyProp0 == 1 ? key : val;
+         }
+ 
+         if (obj instanceof BinaryObject) {
+             BinaryObject obj0 = (BinaryObject) obj;
+             return fieldValue(obj0);
+         }
+         else if (obj instanceof BinaryObjectBuilder) {
+             BinaryObjectBuilder obj0 = (BinaryObjectBuilder)obj;
+ 
+             return obj0.getField(name());
+         }
+         else
+             throw new IgniteCheckedException("Unexpected binary object class [type=" + obj.getClass() + ']');
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+         Object obj = key() ? key : val;
+ 
+         if (obj == null)
+             return;
+ 
+         Object srcObj = obj;
+ 
+         if (!(srcObj instanceof BinaryObjectBuilder))
+             throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+ 
+         if (parent != null)
+             obj = parent.value(key, val);
+ 
+         boolean needsBuild = false;
+ 
+         if (obj instanceof BinaryObjectExImpl) {
+             if (parent == null)
+                 throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+ 
+             needsBuild = true;
+ 
+             obj = ((BinaryObjectExImpl)obj).toBuilder();
+         }
+ 
+         if (!(obj instanceof BinaryObjectBuilder))
+             throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+ 
+         setValue0((BinaryObjectBuilder) obj, propName, propVal, type());
+ 
+         if (needsBuild) {
+             obj = ((BinaryObjectBuilder) obj).build();
+ 
+             assert parent != null;
+ 
+             // And now let's set this newly constructed object to parent
+             setValue0((BinaryObjectBuilder) srcObj, parent.propName, obj, obj.getClass());
+         }
+     }
+ 
+     /**
+      * @param builder Object builder.
+      * @param field Field name.
+      * @param val Value to set.
+      * @param valType Type of {@code val}.
+      * @param <T> Value type.
+      */
+     private <T> void setValue0(BinaryObjectBuilder builder, String field, Object val, Class<T> valType) {
+         //noinspection unchecked
+         builder.setField(field, (T)val, valType);
+     }
+ 
+     /**
+      * Get binary field for the property.
+      *
+      * @param obj Target object.
+      * @return Binary field.
+      */
+     private BinaryField binaryField(BinaryObject obj) {
++        if (ctx.query().skipFieldLookup())
++            return null;
++
+         BinaryField field0 = field;
+ 
+         if (field0 == null && !fieldTaken) {
+             BinaryType type = obj instanceof BinaryObjectEx ? ((BinaryObjectEx)obj).rawType() : obj.type();
+ 
+             if (type != null) {
+                 field0 = type.field(propName);
+ 
+                 assert field0 != null;
+ 
+                 field = field0;
+             }
+ 
+             fieldTaken = true;
+         }
+ 
+         return field0;
+     }
+ 
+     /**
+      * Gets field value for the given binary object.
+      *
+      * @param obj Binary object.
+      * @return Field value.
+      */
+     @SuppressWarnings("IfMayBeConditional")
+     private Object fieldValue(BinaryObject obj) {
+         BinaryField field = binaryField(obj);
+ 
+         if (field != null)
+             return field.value(obj);
+         else
+             return obj.field(propName);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public String name() {
+         return alias;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Class<?> type() {
+         return type;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean key() {
+         int isKeyProp0 = isKeyProp;
+ 
+         if (isKeyProp0 == 0)
+             throw new IllegalStateException("Ownership flag not set for binary property. Have you set 'keyFields'" +
+                 " property of QueryEntity in programmatic or XML configuration?");
+ 
+         return isKeyProp0 == 1;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public GridQueryProperty parent() {
+         return parent;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
index 1fb43db,0000000..909b524
mode 100644,000000..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
@@@ -1,89 -1,0 +1,90 @@@
 +/*
 + * 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.rest.handlers.cluster;
 +
 +import java.util.Collection;
 +
 +import org.apache.ignite.internal.GridKernalContext;
 +import org.apache.ignite.internal.IgniteInternalFuture;
 +import org.apache.ignite.internal.processors.rest.GridRestCommand;
 +import org.apache.ignite.internal.processors.rest.GridRestResponse;
 +import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandlerAdapter;
 +import org.apache.ignite.internal.processors.rest.request.GridRestChangeStateRequest;
 +import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 +import org.apache.ignite.internal.util.future.GridFutureAdapter;
 +import org.apache.ignite.internal.util.typedef.internal.SB;
 +import org.apache.ignite.internal.util.typedef.internal.U;
 +
 +import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_ACTIVE;
 +import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_CURRENT_STATE;
 +import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_INACTIVE;
 +
 +/**
 + *
 + */
 +public class GridChangeStateCommandHandler extends GridRestCommandHandlerAdapter {
 +    /** Commands. */
 +    private static final Collection<GridRestCommand> commands =
 +        U.sealList(CLUSTER_ACTIVE, CLUSTER_INACTIVE, CLUSTER_CURRENT_STATE);
++
 +    /**
 +     * @param ctx Context.
 +     */
 +    public GridChangeStateCommandHandler(GridKernalContext ctx) {
 +        super(ctx);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public Collection<GridRestCommand> supportedCommands() {
 +        return commands;
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest restRest) {
 +        GridRestChangeStateRequest req = (GridRestChangeStateRequest)restRest;
 +
 +        final GridFutureAdapter<GridRestResponse> fut = new GridFutureAdapter<>();
 +
 +        final GridRestResponse res = new GridRestResponse();
 +
 +        try {
 +            if (req.command().equals(CLUSTER_CURRENT_STATE)) {
 +                Boolean currentState = ctx.state().active();
 +
 +                res.setResponse(currentState);
 +            }
 +            else
 +                ctx.state().changeGlobalState(req.active()).get();
 +
 +            fut.onDone(res);
 +        }
 +        catch (Exception e) {
 +            SB sb = new SB();
 +
 +            sb.a(e.getMessage()).a("\n").a("suppressed: \n");
 +
 +            for (Throwable t:e.getSuppressed())
 +                sb.a(t.getMessage()).a("\n");
 +
 +            res.setError(sb.toString());
 +
 +            fut.onDone(res);
 +        }
 +        return fut;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestChangeStateRequest.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestChangeStateRequest.java
index 666d2a9,0000000..8bbfedd
mode 100644,000000..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestChangeStateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestChangeStateRequest.java
@@@ -1,57 -1,0 +1,57 @@@
 +/*
 + * 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.rest.request;
 +
 +/**
 + *
 + */
 +public class GridRestChangeStateRequest extends GridRestRequest {
 +    /** Active. */
 +    private boolean active;
 +
 +    /** Request current state. */
 +    private boolean reqCurrentState;
 +
 +    /**
 +     *
 +     */
 +    public boolean active() {
 +        return active;
 +    }
 +
 +    /**
 +     *
 +     */
 +    public void active(boolean active) {
 +        this.active = active;
 +    }
 +
 +    /**
 +     *
 +     */
 +    public boolean isReqCurrentState() {
 +        return reqCurrentState;
 +    }
 +
 +    /**
 +     *
 +     */
 +    public void reqCurrentState() {
-         this.reqCurrentState = true;
++        reqCurrentState = true;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index cd433f8,e0a5c7c..1900ac7
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@@ -366,32 -366,6 +370,32 @@@ public class GridServiceProcessor exten
      }
  
      /** {@inheritDoc} */
 +    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
 +        if (log.isDebugEnabled())
 +            log.debug("Activate service processor [nodeId=" + ctx.localNodeId() +
 +                " topVer=" + ctx.discovery().topologyVersionEx() + " ]");
 +
-         depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.gridName(), "srvc-deploy"));
++        depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), "srvc-deploy"));
 +
 +        start(true);
 +
 +        onKernalStart(true);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
 +        if (log.isDebugEnabled())
 +            log.debug("DeActivate service processor [nodeId=" + ctx.localNodeId() +
 +                " topVer=" + ctx.discovery().topologyVersionEx() + " ]");
 +
 +        cancelFutures(depFuts, new IgniteCheckedException("Failed to deploy service, cluster in active."));
 +
 +        cancelFutures(undepFuts, new IgniteCheckedException("Failed to undeploy service, cluster in active."));
 +
 +        onKernalStop(true);
 +    }
 +
 +    /** {@inheritDoc} */
      @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
          cancelFutures(depFuts, new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
              "Failed to deploy service, client node disconnected."));

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/session/GridTaskSessionProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/util/worker/GridWorker.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index db97d09,fe915e5..8efb8e2
mode 100644,100755..100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 96467a0,35dcdba..cc07f3e
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@@ -291,15 -292,7 +293,15 @@@ class ServerImpl extends TcpDiscoveryIm
      }
  
      /** {@inheritDoc} */
 +    @Override public int boundPort() throws IgniteSpiException {
 +        if (tcpSrvr == null)
 +            tcpSrvr = new TcpServer();
 +
 +        return tcpSrvr.port;
 +    }
 +
 +    /** {@inheritDoc} */
-     @Override public void spiStart(String gridName) throws IgniteSpiException {
+     @Override public void spiStart(String igniteInstanceName) throws IgniteSpiException {
          synchronized (mux) {
              spiState = DISCONNECTED;
          }

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 55200ae,62a6986..66667aa
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@@ -231,22 -232,12 +232,22 @@@ abstract class TcpDiscoveryImpl 
      public abstract void failNode(UUID nodeId, @Nullable String warning);
  
      /**
-      * @param gridName Grid name.
+      * @param igniteInstanceName Ignite instance name.
       * @throws IgniteSpiException If failed.
       */
-     public abstract void spiStart(@Nullable String gridName) throws IgniteSpiException;
+     public abstract void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException;
  
      /**
 +     * Will start TCP server if applicable and not started yet.
 +     *
 +     * @return Port this instance bound to.
 +     * @throws IgniteSpiException If failed.
 +     */
 +    public int boundPort() throws IgniteSpiException {
 +        return 0;
 +    }
 +
 +    /**
       * @throws IgniteSpiException If failed.
       */
      public abstract void spiStop() throws IgniteSpiException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 4bb8dff,4516f75..e54ee98
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@@ -413,11 -410,8 +414,14 @@@ public class TcpDiscoverySpi extends Ig
      private boolean clientReconnectDisabled;
  
      /** */
 +    private Serializable consistentId;
 +
 +    /** Local node addresses. */
 +    private IgniteBiTuple<Collection<String>, Collection<String>> addrs;
 +
++    /** */
+     protected IgniteSpiContext spiCtx;
+ 
      /** {@inheritDoc} */
      @Override public String getSpiState() {
          return impl.getSpiState();
@@@ -1790,21 -1730,7 +1779,20 @@@
      }
  
      /** {@inheritDoc} */
-     @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+     @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
 +        initializeImpl();
 +
-         registerMBean(gridName, this, TcpDiscoverySpiMBean.class);
++        registerMBean(igniteInstanceName, this, TcpDiscoverySpiMBean.class);
 +
-         impl.spiStart(gridName);
++        impl.spiStart(igniteInstanceName);
 +    }
 +
 +    /**
 +     *
 +     */
 +    private void initializeImpl() {
 +        if (impl != null)
 +            return;
- 
          initFailureDetectionTimeout();
  
          if (!forceSrvMode && (Boolean.TRUE.equals(ignite.configuration().isClientMode()))) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index fbac6b1,d8b1fc1..d778854
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@@ -513,14 -514,14 +514,15 @@@ public class TcpDiscoveryNode extends G
      }
  
      /**
+      * @param nodeAttrs Current node attributes.
       * @return Copy of local node for client reconnect request.
       */
-     public TcpDiscoveryNode clientReconnectNode() {
+     public TcpDiscoveryNode clientReconnectNode(Map<String, Object> nodeAttrs) {
 -        TcpDiscoveryNode node = new TcpDiscoveryNode(id, addrs, hostNames, discPort, metricsProvider, ver,
 -            null);
 +        TcpDiscoveryNode node = new TcpDiscoveryNode(
 +            id, addrs, hostNames, discPort, metricsProvider, ver, null
 +        );
  
-         node.attrs = attrs;
+         node.attrs = Collections.unmodifiableMap(new HashMap<>(nodeAttrs));
          node.clientRouterNodeId = clientRouterNodeId;
  
          return node;

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/java/org/apache/ignite/startup/BasicWarmupClosure.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/cache/store/IgniteCacheExpiryStoreLoadSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/managers/GridNoopManagerSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
index 759179f,9e17449..2038993
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
@@@ -58,15 -57,8 +58,15 @@@ public class GridDeploymentMessageCount
      private Map<String, MessageCountingCommunicationSpi> commSpis = new ConcurrentHashMap8<>();
  
      /** {@inheritDoc} */
 +    @Override protected void afterTestsStopped() throws Exception {
 +        stopAllGrids();
 +
 +        assert G.allGrids().isEmpty();
 +    }
 +
 +    /** {@inheritDoc} */
-     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-         IgniteConfiguration cfg = super.getConfiguration(gridName);
+     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
  
          TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
index 71f1bbd,a2e317a..99133b8
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
@@@ -68,10 -68,11 +68,10 @@@ public class BinaryObjectOffHeapUnswapT
      private CacheAtomicityMode atomicityMode;
  
      /** {@inheritDoc} */
-     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-         IgniteConfiguration c = super.getConfiguration(gridName);
+     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
  
          c.setMarshaller(new BinaryMarshaller());
 -        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
  
          return c;
      }

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNearReaderUpdateTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/EntryVersionConsistencyReadThroughTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index f0b3cf0,0b4d756..b7d7170
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@@ -74,15 -74,11 +74,15 @@@ public abstract class GridAbstractCache
      private static CacheInterceptor<Integer, Integer> interceptor;
  
      /** {@inheritDoc} */
-     @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
-         final IgniteConfiguration cfg = super.getConfiguration(gridName);
+     @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception {
+         final IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
  
 +        cfg.setLateAffinityAssignment(true);
 +
          final CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>(CACHE_NAME);
  
 +        assertNotNull(interceptor);
 +
          ccfg.setInterceptor(interceptor);
          ccfg.setAtomicityMode(atomicityMode());
          ccfg.setWriteSynchronizationMode(FULL_SYNC);

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index d9fe2a8,5aabaae..bda0606
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@@ -99,9 -100,11 +99,9 @@@ public abstract class GridCacheAbstract
  
          ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER).setForceServerMode(true);
  
-         if (testClientNode() && getTestGridName(0).equals(gridName))
+         if (testClientNode() && getTestIgniteInstanceName(0).equals(igniteInstanceName))
              cfg.setClientMode(true);
  
 -        cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 -
          ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
  
          return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index fee01bf,e175fd9..c758ec1
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@@ -213,14 -215,8 +213,14 @@@ public abstract class GridCacheAbstract
  
          cfg.setDiscoverySpi(disco);
  
-         cfg.setCacheConfiguration(cacheConfiguration(gridName));
+         cfg.setCacheConfiguration(cacheConfiguration(igniteInstanceName));
  
 +        TcpCommunicationSpi comm = new TcpCommunicationSpi();
 +
 +        comm.setSharedMemoryPort(-1);
 +
 +        cfg.setCommunicationSpi(comm);
 +
          return cfg;
      }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractTxReadTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreMultithreadedAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
index 545087d,acc4a80..625c969
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
@@@ -39,22 -40,25 +39,22 @@@ public class GridCacheConfigurationVali
      private static final String NON_DFLT_CACHE_NAME = "non-dflt-cache";
  
      /** */
-     private static final String WRONG_PRELOAD_MODE_GRID_NAME = "preloadModeCheckFails";
+     private static final String WRONG_PRELOAD_MODE_IGNITE_INSTANCE_NAME = "preloadModeCheckFails";
  
      /** */
-     private static final String WRONG_CACHE_MODE_GRID_NAME = "cacheModeCheckFails";
+     private static final String WRONG_CACHE_MODE_IGNITE_INSTANCE_NAME = "cacheModeCheckFails";
  
      /** */
-     private static final String WRONG_AFFINITY_GRID_NAME = "cacheAffinityCheckFails";
+     private static final String WRONG_AFFINITY_IGNITE_INSTANCE_NAME = "cacheAffinityCheckFails";
  
      /** */
-     private static final String WRONG_AFFINITY_MAPPER_GRID_NAME = "cacheAffinityMapperCheckFails";
+     private static final String WRONG_AFFINITY_MAPPER_IGNITE_INSTANCE_NAME = "cacheAffinityMapperCheckFails";
  
      /** */
-     private static final String DUP_CACHES_GRID_NAME = "duplicateCachesCheckFails";
 -    private static final String WRONG_OFF_HEAP_IGNITE_INSTANCE_NAME = "cacheOhhHeapCheckFails";
 -
 -    /** */
+     private static final String DUP_CACHES_IGNITE_INSTANCE_NAME = "duplicateCachesCheckFails";
  
      /** */
-     private static final String DUP_DFLT_CACHES_GRID_NAME = "duplicateDefaultCachesCheckFails";
+     private static final String DUP_DFLT_CACHES_IGNITE_INSTANCE_NAME = "duplicateDefaultCachesCheckFails";
  
      /** */
      private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
@@@ -97,18 -101,20 +97,18 @@@
          namedCacheCfg.setAffinity(new RendezvousAffinityFunction());
  
          // Modify cache config according to test parameters.
-         if (gridName.contains(WRONG_PRELOAD_MODE_GRID_NAME))
+         if (igniteInstanceName.contains(WRONG_PRELOAD_MODE_IGNITE_INSTANCE_NAME))
              dfltCacheCfg.setRebalanceMode(SYNC);
-         else if (gridName.contains(WRONG_CACHE_MODE_GRID_NAME))
+         else if (igniteInstanceName.contains(WRONG_CACHE_MODE_IGNITE_INSTANCE_NAME))
              dfltCacheCfg.setCacheMode(REPLICATED);
-         else if (gridName.contains(WRONG_AFFINITY_GRID_NAME))
+         else if (igniteInstanceName.contains(WRONG_AFFINITY_IGNITE_INSTANCE_NAME))
              dfltCacheCfg.setAffinity(new TestRendezvousAffinityFunction());
-         else if (gridName.contains(WRONG_AFFINITY_MAPPER_GRID_NAME))
+         else if (igniteInstanceName.contains(WRONG_AFFINITY_MAPPER_IGNITE_INSTANCE_NAME))
              dfltCacheCfg.setAffinityMapper(new TestCacheDefaultAffinityKeyMapper());
 -        else if (igniteInstanceName.contains(WRONG_OFF_HEAP_IGNITE_INSTANCE_NAME))
 -            dfltCacheCfg.setMemoryMode(OFFHEAP_VALUES);
  
-         if (gridName.contains(DUP_CACHES_GRID_NAME))
+         if (igniteInstanceName.contains(DUP_CACHES_IGNITE_INSTANCE_NAME))
              cfg.setCacheConfiguration(namedCacheCfg, namedCacheCfg);
-         else if (gridName.contains(DUP_DFLT_CACHES_GRID_NAME))
+         else if (igniteInstanceName.contains(DUP_DFLT_CACHES_IGNITE_INSTANCE_NAME))
              cfg.setCacheConfiguration(dfltCacheCfg, dfltCacheCfg);
          else
              // Normal configuration.

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTieredAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheP2PUndeploySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedWritesTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePreloadingEvictionsSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStopSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreValueBytesSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapPreloadSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerEvictionSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheValueBytesPreloadingSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheValueConsistencyAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVersionMultinodeTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
index f0cfdda,6877800..4b1268a
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
@@@ -119,9 -119,10 +119,9 @@@ public abstract class IgniteCacheAbstra
       * @throws Exception In case of error.
       */
      @SuppressWarnings("unchecked")
-     protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+     protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
          CacheConfiguration cfg = defaultCacheConfiguration();
  
 -        cfg.setSwapEnabled(swapEnabled());
          cfg.setCacheMode(cacheMode());
          cfg.setAtomicityMode(atomicityMode());
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/81ae2d83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpireAndUpdateConsistencyTest.java
----------------------------------------------------------------------