You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/25 11:55:19 UTC

[9/9] incubator-ignite git commit: ignite-959 Avoid sending key-value class definitions to servers when starting a cache

ignite-959 Avoid sending key-value class definitions to servers when starting a cache


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

Branch: refs/heads/ignite-959
Commit: 2fcc9c2b6936a237ad79fe92e26d37979550a474
Parents: 7fc5595
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 24 16:04:36 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jun 25 12:54:49 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheTypeMetadata.java  |  72 ++-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  18 +-
 .../processors/query/GridQueryProcessor.java    | 544 ++++++++++++-------
 .../CacheMetricsForClusterGroupSelfTest.java    |   2 +-
 5 files changed, 421 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2fcc9c2b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
index 20129b7..de57346 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cache;
 
 import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 
@@ -43,6 +44,12 @@ public class CacheTypeMetadata implements Serializable {
     /** Value class used to store value in cache. */
     private String valType;
 
+    /** Key type name. */
+    private String keyTypeName;
+
+    /** Value type name. */
+    private String valTypeName;
+
     /** Key fields. */
     @GridToStringInclude
     private Collection<CacheTypeFieldMetadata> keyFields;
@@ -71,6 +78,10 @@ public class CacheTypeMetadata implements Serializable {
     @GridToStringInclude
     private Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps;
 
+    /** Aliases for fields. */
+    @GridToStringInclude
+    private Map<String, String> aliases;
+
     /**
      * Default constructor.
      */
@@ -88,6 +99,8 @@ public class CacheTypeMetadata implements Serializable {
         txtFlds = new LinkedHashSet<>();
 
         grps = new LinkedHashMap<>();
+
+        aliases = new HashMap<>();
     }
 
     /**
@@ -115,6 +128,8 @@ public class CacheTypeMetadata implements Serializable {
         txtFlds = new LinkedHashSet<>(src.getTextFields());
 
         grps = new LinkedHashMap<>(src.getGroups());
+
+        aliases = new HashMap<>(src.aliases);
     }
 
     /**
@@ -178,6 +193,7 @@ public class CacheTypeMetadata implements Serializable {
      */
     public void setKeyType(Class<?> cls) {
         setKeyType(cls.getName());
+        keyTypeName = typeName(cls);
     }
 
     /**
@@ -204,7 +220,17 @@ public class CacheTypeMetadata implements Serializable {
      * @param cls Value type class.
      */
     public void setValueType(Class<?> cls) {
-        setValueType(cls.getName());
+        setValueType(typeName(cls));
+        valTypeName = typeName(cls);
+    }
+
+    /**
+     * Gets value type name.
+     *
+     * @return Value type name.
+     */
+    public String getValTypeName() {
+        return valTypeName;
     }
 
     /**
@@ -333,8 +359,52 @@ public class CacheTypeMetadata implements Serializable {
         this.grps = grps;
     }
 
+    /**
+     * Returns fields aliases.
+     *
+     * @return fields aliases.
+     */
+    public Map<String, String> getAliases() {
+        return aliases;
+    }
+
+    /**
+     * Adds field alias.
+     *
+     * @param alias Alias.
+     * @param fieldName Field name.
+     */
+    public void addAlias(String alias, String fieldName) {
+        A.notNull(alias, "alias");
+        A.notNull(fieldName, "fieldName");
+
+        aliases.put(alias, fieldName);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheTypeMetadata.class, this);
     }
+
+    /**
+     * @param cls Class.
+     */
+    private 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;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2fcc9c2b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 7af4974..4ec3e4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -298,7 +298,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private boolean sqlEscapeAll;
 
     /** */
-    private Class<?>[] indexedTypes;
+    private transient Class<?>[] indexedTypes;
 
     /** */
     private int sqlOnheapRowCacheSize = DFLT_SQL_ONHEAP_ROW_CACHE_SIZE;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2fcc9c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 2f7f22c..428063b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -583,6 +583,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             // Initialize defaults.
             initialize(cfg, cacheObjCtx);
 
+            ctx.query().generateTypeMetadata(cfg);
+
             cfgs[i] = cfg; // Replace original configuration value.
 
             String masked = maskNull(cfg.getName());
@@ -594,8 +596,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
                         "assign unique name to each cache): " + U.maskName(cacheName));
                 else
-                    throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
-                        "assign unique name to each cache).");
+                    throw new IgniteCheckedException("Default cache has already been configured " +
+                        "(check configuration and assign unique name to each cache).");
             }
 
             CacheType cacheType;
@@ -691,7 +693,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         if (rmtCfg != null) {
                             CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                            checkCache(locCfg, rmtCfg, n, desc);
+                            checkCache(locCfg, rmtCfg, n);
 
                             // Check plugin cache configurations.
                             CachePluginManager pluginMgr = desc.pluginManager();
@@ -1625,7 +1627,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         for (DynamicCacheDescriptor desc : registeredCaches.values()) {
             if (!desc.cancelled()) {
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+                DynamicCacheChangeRequest req =
+                    new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
                 req.startCacheConfiguration(desc.cacheConfiguration());
 
@@ -1930,6 +1933,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 try {
                     CacheConfiguration cfg = new CacheConfiguration(ccfg);
 
+                    ctx.query().generateTypeMetadata(cfg);
+
                     CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
 
                     initialize(cfg, cacheObjCtx);
@@ -2260,11 +2265,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param locCfg Local configuration.
      * @param rmtCfg Remote configuration.
      * @param rmtNode Remote node.
-     * @param desc Cache descriptor.
      * @throws IgniteCheckedException If check failed.
      */
-    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode,
-        DynamicCacheDescriptor desc) throws IgniteCheckedException {
+    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode)
+        throws IgniteCheckedException {
         ClusterNode locNode = ctx.discovery().localNode();
 
         UUID rmt = rmtNode.id();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2fcc9c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e080c6d..a8fe60e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -121,102 +121,341 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         idx.registerCache(ccfg);
 
         try {
-            if (!F.isEmpty(ccfg.getTypeMetadata())) {
-                for (CacheTypeMetadata meta : ccfg.getTypeMetadata()) {
-                    if (F.isEmpty(meta.getValueType()))
-                        throw new IgniteCheckedException("Value type is not set: " + meta);
+            if (!F.isEmpty(ccfg.getTypeMetadata()))
+                processTypeMetadata(ccfg);
+        }
+        catch (IgniteCheckedException | RuntimeException e) {
+            idx.unregisterCache(ccfg);
 
-                    TypeDescriptor desc = new TypeDescriptor();
+            throw e;
+        }
+    }
 
-                    Class<?> valCls = U.classForName(meta.getValueType(), null);
+    /**
+     * Generates cache type metadata if indexed types are defined.
+     *
+     * @param ccfg Cache configuration.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public void generateTypeMetadata(CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
+        if (ccfg.getTypeMetadata() == null) {
+            Class<?>[] clss = ccfg.getIndexedTypes();
 
-                    desc.name(valCls != null ? typeName(valCls) : meta.getValueType());
+            if (!F.isEmpty(clss)) {
+                List<CacheTypeMetadata> metadata = new ArrayList<>(clss.length / 2);
 
-                    desc.valueClass(valCls != null ? valCls : Object.class);
-                    desc.keyClass(
-                        meta.getKeyType() == null ?
-                            Object.class :
-                            U.classForName(meta.getKeyType(), Object.class));
+                for (int i = 0; i < clss.length; i += 2) {
+                    Class<?> keyCls = clss[i];
+                    Class<?> valCls = clss[i + 1];
 
-                    TypeId typeId;
+                    CacheTypeMetadata meta = new CacheTypeMetadata();
 
-                    if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
-                        processPortableMeta(meta, desc);
+                    meta.setKeyType(keyCls);
+                    meta.setValueType(valCls);
 
-                        typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
-                    }
-                    else {
-                        processClassMeta(meta, desc);
+                    Map<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> orderedGroups = new HashMap<>();
 
-                        typeId = new TypeId(ccfg.getName(), valCls);
-                    }
+                    processClassAnnotations(keyCls, meta, null, orderedGroups);
+                    processClassAnnotations(valCls, meta, null, orderedGroups);
 
-                    addTypeByName(ccfg, desc);
-                    types.put(typeId, desc);
+                    fillOrderedGroups(meta, orderedGroups);
 
-                    desc.registered(idx.registerType(ccfg.getName(), desc));
+                    metadata.add(meta);
                 }
+
+                ccfg.setTypeMetadata(metadata);
             }
+        }
 
-            Class<?>[] clss = ccfg.getIndexedTypes();
+        if (ctx.cacheObjects().isPortableEnabled() && ccfg.getTypeMetadata() != null)
+            maskClasses(ccfg.getTypeMetadata());
+    }
 
-            if (!F.isEmpty(clss)) {
-                for (int i = 0; i < clss.length; i += 2) {
-                    Class<?> keyCls = clss[i];
-                    Class<?> valCls = clss[i + 1];
+    /**
+     * @param metadata Metadata.
+     */
+    private static void maskClasses(Collection<CacheTypeMetadata> metadata) {
+        for (CacheTypeMetadata meta : metadata) {
+            maskFieldsTypes(meta.getQueryFields());
+
+            maskFieldsTypes(meta.getAscendingFields());
 
-                    TypeDescriptor desc = processKeyAndValueClasses(keyCls, valCls);
+            maskFieldsTypes(meta.getDescendingFields());
 
-                    addTypeByName(ccfg, desc);
-                    types.put(new TypeId(ccfg.getName(), valCls), desc);
+            for (LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> grp : meta.getGroups().values()) {
+                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> entry : grp.entrySet()) {
+                    Class<?> cls = entry.getValue().get1();
 
-                    desc.registered(idx.registerType(ccfg.getName(), desc));
+                    if (!U.isJdk(cls))
+                        entry.getValue().set1(Object.class);
                 }
             }
         }
-        catch (IgniteCheckedException | RuntimeException e) {
-            idx.unregisterCache(ccfg);
+    }
 
-            throw e;
+    /**
+     * @param fields Fields.
+     */
+    private static void maskFieldsTypes(Map<String, Class<?>> fields) {
+        for (Map.Entry<String, Class<?>> entry : fields.entrySet())
+            if (!U.isJdk(entry.getValue()))
+                entry.setValue(Objects.class);
+    }
+
+    /**
+     * @param meta Cache type metadata.
+     * @param orderedGroups Ordered groups.
+     * @throws IgniteCheckedException In case or error.
+     */
+    private static void fillOrderedGroups(CacheTypeMetadata meta,
+        Map<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> orderedGroups)
+        throws IgniteCheckedException
+    {
+        for (Map.Entry<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> ordGrp : orderedGroups.entrySet()) {
+            String grpName = ordGrp.getKey();
+
+            Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> groups = meta.getGroups();
+
+            LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> fields = groups.get(grpName);
+
+            if (fields == null)
+                groups.put(grpName, fields = new LinkedHashMap<>());
+
+            for (Map.Entry<Integer, T3<String, Class<?>, Boolean>> grpFields : ordGrp.getValue().entrySet()) {
+                String name = grpFields.getValue().get1();
+
+                Class<?> cls = grpFields.getValue().get2();
+
+                Boolean desc = grpFields.getValue().get3();
+
+                if (fields.put(name, new T2<Class<?>, Boolean>(cls, desc)) != null)
+                    throw new IgniteCheckedException("Field " + name + " already exists in group " + grpName);
+            }
         }
     }
 
     /**
-     * @param ccfg Cache configuration.
-     * @param desc Type descriptor.
-     * @throws IgniteCheckedException If failed.
+     * @param cls Class.
+     * @param meta Type metadata.
+     * @param parentField Parent field name.
      */
-    private void addTypeByName(CacheConfiguration<?,?> ccfg, TypeDescriptor desc) throws IgniteCheckedException {
-        if (typesByName.putIfAbsent(new TypeName(ccfg.getName(), desc.name()), desc) != null)
-            throw new IgniteCheckedException("Type with name '" + desc.name() + "' already indexed " +
-                "in cache '" + ccfg.getName() + "'.");
+    private void processClassAnnotations(Class<?> cls, CacheTypeMetadata meta,
+        String parentField, Map<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> orderedGroups)
+        throws IgniteCheckedException
+    {
+        if (U.isJdk(cls) || idx.isGeometryClass(cls))
+            return;
+
+        for (Class<?> c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
+            for (Field field : c.getDeclaredFields()) {
+                QueryTextField txtAnn = field.getAnnotation(QueryTextField.class);
+
+                if (txtAnn != null) {
+                    String fieldName = parentField == null ? field.getName() : parentField + '.' + field.getName();
+
+                    meta.getTextFields().add(fieldName);
+
+                    meta.getQueryFields().put(fieldName, field.getType());
+                }
+
+                QuerySqlField sqlAnn = field.getAnnotation(QuerySqlField.class);
+
+                if (sqlAnn != null) {
+                    String name = sqlAnn.name().isEmpty() ? field.getName() : sqlAnn.name();
+
+                    String pathStr = parentField == null ? name : parentField + '.' + name;
+
+                    processClassAnnotations(field.getType(), meta, pathStr, orderedGroups);
+
+                    if (sqlAnn.index()) {
+                        Map<String, Class<?>> fields =
+                            sqlAnn.descending() ? meta.getDescendingFields() : meta.getAscendingFields();
+
+                        fields.put(pathStr, field.getType());
+                    }
+
+                    meta.getQueryFields().put(pathStr, field.getType());
+
+                    if (!sqlAnn.name().isEmpty())
+                        meta.addAlias(sqlAnn.name(), field.getName());
+
+                    if (!F.isEmpty(sqlAnn.groups())) {
+                        for (String grp : sqlAnn.groups()) {
+                            LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> fields = meta.getGroups().get(grp);
+
+                            if (fields == null)
+                                meta.getGroups().put(grp, fields = new LinkedHashMap<>());
+
+                            IgniteBiTuple<Class<?>, Boolean> fieldType =
+                                new IgniteBiTuple<Class<?>, Boolean>(field.getType(), false);
+
+                            if (fields.put(pathStr, fieldType) != null)
+                                throw new IgniteCheckedException("Field " + pathStr +
+                                    " already exists in group " + grp);
+                        }
+                    }
+
+                    if (!F.isEmpty(sqlAnn.orderedGroups())) {
+                        for (QuerySqlField.Group idx : sqlAnn.orderedGroups()) {
+                            TreeMap<Integer, T3<String, Class<?>, Boolean>> orderedFields =
+                                orderedGroups.get(idx.name());
+
+                            if (orderedFields == null)
+                                orderedGroups.put(idx.name(), orderedFields = new TreeMap<>());
+
+                            T3<String, Class<?>, Boolean> grp =
+                                new T3<String, Class<?>, Boolean>(pathStr, field.getType(), idx.descending());
+
+                            if (orderedFields.put(idx.order(), grp) != null)
+                                throw new IgniteCheckedException("Field " + pathStr + " has duplicated order " +
+                                    idx.order() + " in group " + idx.name());
+                        }
+                    }
+                }
+            }
+
+            for (Method mtd : c.getDeclaredMethods()) {
+                String mtdName = mtd.getName().startsWith("get") && mtd.getName().length() > 3 ?
+                    mtd.getName().substring(3) : mtd.getName();
+
+                QuerySqlField sqlAnn = mtd.getAnnotation(QuerySqlField.class);
+                QueryTextField txtAnn = mtd.getAnnotation(QueryTextField.class);
+
+                if (sqlAnn != null || txtAnn != null) {
+                    if (mtd.getParameterTypes().length != 0)
+                        throw new IgniteCheckedException("Getter with QuerySqlField " +
+                            "annotation cannot have parameters: " + mtd);
+
+                    Class<?> type = mtd.getReturnType();
+
+                    if (txtAnn != null) {
+                        String pathStr = parentField == null ? mtdName : parentField + '.' + mtdName;
+
+                        meta.getTextFields().add(pathStr);
+
+                        meta.getQueryFields().put(pathStr, type);
+                    }
+
+                    if (sqlAnn != null) {
+                        String name = sqlAnn.name().isEmpty() ? mtdName : sqlAnn.name();
+
+                        name = parentField == null ? name : parentField + '.' + name;
+
+                        processClassAnnotations(mtd.getReturnType(), meta, name, orderedGroups);
+
+                        if (sqlAnn.index()) {
+                            Map<String, Class<?>> fields =
+                                sqlAnn.descending() ? meta.getDescendingFields() : meta.getAscendingFields();
+
+                            fields.put(name, type);
+                        }
+
+                        meta.getQueryFields().put(name, type);
+
+                        if (!sqlAnn.name().isEmpty())
+                            meta.addAlias(sqlAnn.name(), mtdName);
+
+                        if (!F.isEmpty(sqlAnn.groups())) {
+                            for (String grp : sqlAnn.groups()) {
+                                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> fields =
+                                    meta.getGroups().get(grp);
+
+                                if (fields == null) {
+                                    fields = new LinkedHashMap<>();
+
+                                    meta.getGroups().put(grp, fields);
+                                }
+
+                                IgniteBiTuple<Class<?>, Boolean> fieldType =
+                                    new IgniteBiTuple<Class<?>, Boolean>(mtd.getReturnType(), false);
+
+                                if (fields.put(name, fieldType) != null)
+                                    throw new IgniteCheckedException("Field " + name +
+                                        " already exists in group " + grp);
+                            }
+                        }
+
+                        if (!F.isEmpty(sqlAnn.orderedGroups())) {
+                            for (QuerySqlField.Group idx : sqlAnn.orderedGroups()) {
+                                TreeMap<Integer, T3<String, Class<?>, Boolean>> orderedFields =
+                                    orderedGroups.get(idx.name());
+
+                                if (orderedFields == null)
+                                    orderedGroups.put(idx.name(), orderedFields = new TreeMap<>());
+
+                                T3<String, Class<?>, Boolean> grp =
+                                    new T3<String, Class<?>, Boolean>(name, mtd.getReturnType(), idx.descending());
+
+                                if (orderedFields.put(idx.order(), grp) != null)
+                                    throw new IgniteCheckedException("Field " + name + " has duplicated order " +
+                                        idx.order() + " in group " + idx.name());
+                            }
+                        }
+                    }
+                }
+            }
+        }
     }
 
     /**
-     * @param keyCls Key class.
-     * @param valCls Value class.
-     * @return Type descriptor.
-     * @throws IgniteCheckedException If failed.
+     * @param ccfg Cache configuration.
      */
-    private TypeDescriptor processKeyAndValueClasses(
-        Class<?> keyCls,
-        Class<?> valCls
-    )
-        throws IgniteCheckedException {
-        TypeDescriptor d = new TypeDescriptor();
+    private void processTypeMetadata(CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
+        for (CacheTypeMetadata meta : ccfg.getTypeMetadata()) {
+            if (F.isEmpty(meta.getValueType()))
+                throw new IgniteCheckedException("Value type is not set: " + meta);
+
+            TypeDescriptor desc = new TypeDescriptor();
+
+            Class<?> valCls = U.classForName(meta.getValueType(), null);
+
+            //desc.name(valCls != null ? typeName(valCls) : meta.getValueType());
+            desc.name(meta.getValTypeName());
+
+            desc.valueClass(valCls != null ? valCls : Object.class);
+            desc.keyClass(
+                meta.getKeyType() == null ?
+                    Object.class :
+                    U.classForName(meta.getKeyType(), Object.class));
+
+            TypeId typeId;
+
+            if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
+                processPortableMeta(meta, desc);
 
-        d.keyClass(keyCls);
-        d.valueClass(valCls);
+                typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
+            }
+            else {
+                processClassMeta(meta, desc);
 
-        processAnnotationsInClass(true, d.keyCls, d, null);
+                typeId = new TypeId(ccfg.getName(), valCls);
 
-        String valTypeName = typeName(valCls);
+                // We have to index primitive _val.
+                if ((U.isJdk(valCls) || idx.isGeometryClass(valCls)) && idx.isSqlType(valCls)) {
+                    String idxName = "_val_idx";
 
-        d.name(valTypeName);
+                    desc.addIndex(idxName, idx.isGeometryClass(valCls) ? GEO_SPATIAL : SORTED);
+
+                    desc.addFieldToIndex(idxName, "_VAL", 0, false);
+                }
+            }
 
-        processAnnotationsInClass(false, d.valCls, d, null);
+            addTypeByName(ccfg, desc);
+            types.put(typeId, desc);
 
-        return d;
+            desc.registered(idx.registerType(ccfg.getName(), desc));
+        }
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @param desc Type descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void addTypeByName(CacheConfiguration<?,?> ccfg, TypeDescriptor desc) throws IgniteCheckedException {
+        if (typesByName.putIfAbsent(new TypeName(ccfg.getName(), desc.name()), desc) != null)
+            throw new IgniteCheckedException("Type with name '" + desc.name() + "' already indexed " +
+                "in cache '" + ccfg.getName() + "'.");
     }
 
     /** {@inheritDoc} */
@@ -1006,130 +1245,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Process annotations for class.
-     *
-     * @param key If given class relates to key.
-     * @param cls Class.
-     * @param type Type descriptor.
-     * @param parent Parent in case of embeddable.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void processAnnotationsInClass(boolean key, Class<?> cls, TypeDescriptor type,
-        @Nullable ClassProperty parent) throws IgniteCheckedException {
-        if (U.isJdk(cls) || idx.isGeometryClass(cls)) {
-            if (parent == null && !key && idx.isSqlType(cls) ) { // We have to index primitive _val.
-                String idxName = "_val_idx";
-
-                type.addIndex(idxName, idx.isGeometryClass(cls) ? GEO_SPATIAL : SORTED);
-
-                type.addFieldToIndex(idxName, "_VAL", 0, false);
-            }
-
-            return;
-        }
-
-        if (parent != null && parent.knowsClass(cls))
-            throw new IgniteCheckedException("Recursive reference found in type: " + cls.getName());
-
-        if (parent == null) { // Check class annotation at top level only.
-            QueryTextField txtAnnCls = cls.getAnnotation(QueryTextField.class);
-
-            if (txtAnnCls != null)
-                type.valueTextIndex(true);
-
-            QueryGroupIndex grpIdx = cls.getAnnotation(QueryGroupIndex.class);
-
-            if (grpIdx != null)
-                type.addIndex(grpIdx.name(), SORTED);
-
-            QueryGroupIndex.List grpIdxList = cls.getAnnotation(QueryGroupIndex.List.class);
-
-            if (grpIdxList != null && !F.isEmpty(grpIdxList.value())) {
-                for (QueryGroupIndex idx : grpIdxList.value())
-                    type.addIndex(idx.name(), SORTED);
-            }
-        }
-
-        for (Class<?> c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
-            for (Field field : c.getDeclaredFields()) {
-                QuerySqlField sqlAnn = field.getAnnotation(QuerySqlField.class);
-                QueryTextField txtAnn = field.getAnnotation(QueryTextField.class);
-
-                if (sqlAnn != null || txtAnn != null) {
-                    ClassProperty prop = new ClassProperty(field, key);
-
-                    prop.parent(parent);
-
-                    processAnnotation(key, sqlAnn, txtAnn, field.getType(), prop, type);
-
-                    type.addProperty(prop, true);
-                }
-            }
-
-            for (Method mtd : c.getDeclaredMethods()) {
-                QuerySqlField sqlAnn = mtd.getAnnotation(QuerySqlField.class);
-                QueryTextField txtAnn = mtd.getAnnotation(QueryTextField.class);
-
-                if (sqlAnn != null || txtAnn != null) {
-                    if (mtd.getParameterTypes().length != 0)
-                        throw new IgniteCheckedException("Getter with QuerySqlField " +
-                            "annotation cannot have parameters: " + mtd);
-
-                    ClassProperty prop = new ClassProperty(mtd, key);
-
-                    prop.parent(parent);
-
-                    processAnnotation(key, sqlAnn, txtAnn, mtd.getReturnType(), prop, type);
-
-                    type.addProperty(prop, true);
-                }
-            }
-        }
-    }
-
-    /**
-     * Processes annotation at field or method.
-     *
-     * @param key If given class relates to key.
-     * @param sqlAnn SQL annotation, can be {@code null}.
-     * @param txtAnn H2 text annotation, can be {@code null}.
-     * @param cls Class of field or return type for method.
-     * @param prop Current property.
-     * @param desc Class description.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void processAnnotation(boolean key, QuerySqlField sqlAnn, QueryTextField txtAnn,
-        Class<?> cls, ClassProperty prop, TypeDescriptor desc) throws IgniteCheckedException {
-        if (sqlAnn != null) {
-            processAnnotationsInClass(key, cls, desc, prop);
-
-            if (!sqlAnn.name().isEmpty())
-                prop.name(sqlAnn.name());
-
-            if (sqlAnn.index()) {
-                String idxName = prop.name() + "_idx";
-
-                desc.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
-
-                desc.addFieldToIndex(idxName, prop.name(), 0, sqlAnn.descending());
-            }
-
-            if (!F.isEmpty(sqlAnn.groups())) {
-                for (String group : sqlAnn.groups())
-                    desc.addFieldToIndex(group, prop.name(), 0, false);
-            }
-
-            if (!F.isEmpty(sqlAnn.orderedGroups())) {
-                for (QuerySqlField.Group idx : sqlAnn.orderedGroups())
-                    desc.addFieldToIndex(idx.name(), prop.name(), idx.order(), idx.descending());
-            }
-        }
-
-        if (txtAnn != null)
-            desc.addFieldToTextIndex(prop.name());
-    }
-
-    /**
      * Processes declarative metadata for class.
      *
      * @param meta Type metadata.
@@ -1144,12 +1259,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         assert keyCls != null;
         assert valCls != null;
 
-        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
+        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
             ClassProperty prop = buildClassProperty(
                 keyCls,
                 valCls,
                 entry.getKey(),
-                entry.getValue());
+                entry.getValue(),
+                meta.getAliases());
+
+            d.addProperty(prop, false);
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
+            String name = entry.getKey();
+
+            ClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                name,
+                entry.getValue(),
+                meta.getAliases());
 
             d.addProperty(prop, false);
 
@@ -1165,7 +1294,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 keyCls,
                 valCls,
                 entry.getKey(),
-                entry.getValue());
+                entry.getValue(),
+                meta.getAliases());
 
             d.addProperty(prop, false);
 
@@ -1181,7 +1311,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 keyCls,
                 valCls,
                 txtIdx,
-                String.class);
+                String.class,
+                meta.getAliases());
 
             d.addProperty(prop, false);
 
@@ -1203,7 +1334,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         keyCls,
                         valCls,
                         idxField.getKey(),
-                        idxField.getValue().get1());
+                        idxField.getValue().get1(),
+                        meta.getAliases());
 
                     d.addProperty(prop, false);
 
@@ -1215,16 +1347,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
             }
         }
-
-        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(
-                keyCls,
-                valCls,
-                entry.getKey(),
-                entry.getValue());
-
-            d.addProperty(prop, false);
-        }
     }
 
     /**
@@ -1327,16 +1449,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Class property.
      * @throws IgniteCheckedException If failed.
      */
-    private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType)
-        throws IgniteCheckedException {
+    private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr,
+        Class<?> resType, Map<String, String> aliases) throws IgniteCheckedException {
         ClassProperty res = buildClassProperty(
             true,
             keyCls,
             pathStr,
-            resType);
+            resType,
+            aliases);
 
         if (res == null) // We check key before value consistently with PortableProperty.
-            res = buildClassProperty(false, valCls, pathStr, resType);
+            res = buildClassProperty(false, valCls, pathStr, resType, aliases);
 
         if (res == null)
             throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' for " +
@@ -1354,8 +1477,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Property instance corresponding to the given path.
      * @throws IgniteCheckedException If property cannot be created.
      */
-    static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType)
-        throws IgniteCheckedException {
+    static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr,
+        Class<?> resType, Map<String, String> aliases) throws IgniteCheckedException {
         String[] path = pathStr.split("\\.");
 
         ClassProperty res = null;
@@ -1363,10 +1486,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         for (String prop : path) {
             ClassProperty tmp;
 
+            String fieldName = aliases.get(prop);
+
+            if (fieldName == null)
+                fieldName = prop;
+
             try {
                 StringBuilder bld = new StringBuilder("get");
 
-                bld.append(prop);
+                bld.append(fieldName);
 
                 bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
 
@@ -1374,13 +1502,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
             catch (NoSuchMethodException ignore) {
                 try {
-                    tmp = new ClassProperty(cls.getDeclaredField(prop), key);
+                    tmp = new ClassProperty(cls.getDeclaredField(fieldName), key);
                 }
                 catch (NoSuchFieldException ignored) {
                     return null;
                 }
             }
 
+            tmp.name(prop);
+
             tmp.parent(res);
 
             cls = tmp.type();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2fcc9c2b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
index 1ba24e3..455e306 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
@@ -99,7 +99,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
         Collection<ClusterNode> nodes = grid(0).cluster().forRemotes().nodes();
 
         for (ClusterNode node : nodes) {
-            Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode) node).cacheMetrics();
+            Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode)node).cacheMetrics();
             assertNotNull(metrics);
             assertFalse(metrics.isEmpty());
         }