You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/11/10 13:21:46 UTC

[6/9] ignite git commit: IGNITE-1847: Decoupled metadata and binary type.

IGNITE-1847: Decoupled metadata and binary type.


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

Branch: refs/heads/ignite-1847
Commit: efe211b0512efd33acb44adc362a95b2bd60229a
Parents: cf10861
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 10 11:31:40 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 10 11:31:40 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryMetaDataImpl.java   |  42 ++++--
 .../internal/portable/BinaryObjectImpl.java     |   8 +-
 .../portable/BinaryObjectOffheapImpl.java       |   8 +-
 .../internal/portable/BinaryTypeImpl.java       |  81 +++++++++++
 .../internal/portable/PortableContext.java      |  22 ++-
 .../CacheObjectBinaryProcessorImpl.java         | 136 +++++++++++--------
 .../platform/PlatformContextImpl.java           |   2 +-
 7 files changed, 211 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java
index 6231741..a09e14a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java
@@ -27,13 +27,12 @@ import java.util.Map;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.binary.BinaryType;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Portable meta data implementation.
+ * Portable metadata which is passed over a wire.
  */
-public class BinaryMetaDataImpl implements BinaryType, Externalizable {
+public class BinaryMetaDataImpl implements Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -82,35 +81,54 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
         return typeId;
     }
 
-    /** {@inheritDoc} */
-    @Override public String typeName() {
+    /**
+     * @return Type name.
+     */
+    public String typeName() {
         return typeName;
     }
 
-    /** {@inheritDoc} */
-    @Override public Collection<String> fields() {
+    /**
+     * @return Fields.
+     */
+    public Collection<String> fields() {
         return fields != null ? fields.keySet() : Collections.<String>emptyList();
     }
 
     /**
      * @return Fields.
      */
-    public Map<String, Integer> fields0() {
+    public Map<String, Integer> fieldsMap() {
         return fields != null ? fields : Collections.<String, Integer>emptyMap();
     }
 
-    /** {@inheritDoc} */
-    @Nullable @Override public String fieldTypeName(String fieldName) {
+    /**
+     * @param fieldName Field name.
+     * @return Field type name.
+     */
+    @Nullable public String fieldTypeName(String fieldName) {
         Integer typeId = fields != null ? fields.get(fieldName) : null;
 
         return typeId != null ? PortableUtils.fieldTypeName(typeId) : null;
     }
 
-    /** {@inheritDoc} */
-    @Nullable @Override public String affinityKeyFieldName() {
+    /**
+     * @return Affinity key field name.
+     */
+    @Nullable public String affinityKeyFieldName() {
         return affKeyFieldName;
     }
 
+    /**
+     * Wrap metadata into binary type.
+     *
+     * @param ctx Portable context.
+     * @return Binary type.
+     */
+    public BinaryTypeImpl wrap(PortableContext ctx) {
+        return new BinaryTypeImpl(ctx, this);
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(typeId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index 800ca40..7459d24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -465,13 +465,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
     @Override public BinaryField fieldDescriptor(String fieldName) throws BinaryObjectException {
         A.notNull(fieldName, "fieldName");
 
-        int typeId = typeId();
-
-        PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
-
-        int fieldId = ctx.userTypeIdMapper(typeId).fieldId(typeId, fieldName);
-
-        return new BinaryFieldImpl(schemaReg, fieldName, fieldId);
+        return ctx.createField(typeId(), fieldName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index 9b6735f..3c1595c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -143,13 +143,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
     @Override public BinaryField fieldDescriptor(String fieldName) throws BinaryObjectException {
         A.notNull(fieldName, "fieldName");
 
-        int typeId = typeId();
-
-        PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
-
-        int fieldId = ctx.userTypeIdMapper(typeId).fieldId(typeId, fieldName);
-
-        return new BinaryFieldImpl(schemaReg, fieldName, fieldId);
+        return ctx.createField(typeId(), fieldName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
new file mode 100644
index 0000000..d1af2fe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
@@ -0,0 +1,81 @@
+/*
+ * 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.portable;
+
+import org.apache.ignite.binary.BinaryType;
+
+import java.util.Collection;
+
+/**
+ * Binary type implementation.
+ */
+public class BinaryTypeImpl implements BinaryType {
+    /** Portable context. */
+    private final PortableContext ctx;
+
+    /** Type metadata. */
+    private final BinaryMetaDataImpl meta;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Portable context.
+     * @param meta Type  metadata.
+     */
+    public BinaryTypeImpl(PortableContext ctx, BinaryMetaDataImpl meta) {
+        this.ctx = ctx;
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String typeName() {
+        return meta.typeName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> fields() {
+        return meta.fields();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String fieldTypeName(String fieldName) {
+        return meta.fieldTypeName(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String affinityKeyFieldName() {
+        return meta.affinityKeyFieldName();
+    }
+
+    /**
+     * @return Metadata.
+     */
+    public BinaryMetaDataImpl metadata() {
+        return meta;
+    }
+
+    /**
+     * Create field.
+     *
+     * @param fieldName Field name.
+     * @return Field.
+     */
+    public BinaryFieldImpl field(String fieldName) {
+        return ctx.createField(meta.typeId(), fieldName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 9e92f3b..d54e9b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -165,8 +165,9 @@ public class PortableContext implements Externalizable {
      * @param metaHnd Meta data handler.
      * @param igniteCfg Ignite configuration.
      */
-    public PortableContext(PortableMetaDataHandler metaHnd, @Nullable IgniteConfiguration igniteCfg) {
+    public PortableContext(PortableMetaDataHandler metaHnd, IgniteConfiguration igniteCfg) {
         assert metaHnd != null;
+        assert igniteCfg != null;
 
         this.metaHnd = metaHnd;
         this.igniteCfg = igniteCfg;
@@ -776,7 +777,22 @@ public class PortableContext implements Externalizable {
             descByCls.put(cls, desc);
         }
 
-        metaHnd.addMeta(id, new BinaryMetaDataImpl(id, typeName, fieldsMeta, affKeyFieldName));
+        metaHnd.addMeta(id, new BinaryMetaDataImpl(id, typeName, fieldsMeta, affKeyFieldName).wrap(this));
+    }
+
+    /**
+     * Create binary field.
+     *
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Binary field.
+     */
+    public BinaryFieldImpl createField(int typeId, String fieldName) {
+        PortableSchemaRegistry schemaReg = schemaRegistry(typeId);
+
+        int fieldId = userTypeIdMapper(typeId).fieldId(typeId, fieldName);
+
+        return new BinaryFieldImpl(schemaReg, fieldName, fieldId);
     }
 
     /**
@@ -825,7 +841,7 @@ public class PortableContext implements Externalizable {
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
     public void updateMetaData(int typeId, BinaryMetaDataImpl meta) throws BinaryObjectException {
-        metaHnd.addMeta(typeId, meta);
+        metaHnd.addMeta(typeId, meta.wrap(this));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 4ad0762..25ffbdf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.portable.BinaryMetaDataImpl;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
+import org.apache.ignite.internal.portable.BinaryTypeImpl;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableMetaDataHandler;
@@ -66,6 +67,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.jetbrains.annotations.Nullable;
@@ -108,10 +110,10 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private final boolean clientNode;
 
     /** */
-    private volatile IgniteCacheProxy<PortableMetaDataKey, BinaryType> metaDataCache;
+    private volatile IgniteCacheProxy<PortableMetaDataKey, BinaryMetaDataImpl> metaDataCache;
 
     /** */
-    private final ConcurrentHashMap8<PortableMetaDataKey, BinaryType> clientMetaDataCache;
+    private final ConcurrentHashMap8<Integer, BinaryTypeImpl> clientMetaDataCache;
 
     /** Predicate to filter portable meta data in utility cache. */
     private final CacheEntryPredicate metaPred = new CacheEntryPredicateAdapter() {
@@ -136,7 +138,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private IgniteBinary portables;
 
     /** Metadata updates collected before metadata cache is initialized. */
-    private final Map<Integer, BinaryType> metaBuf = new ConcurrentHashMap<>();
+    private final Map<Integer, BinaryMetaDataImpl> metaBuf = new ConcurrentHashMap<>();
 
     /** */
     private UUID metaCacheQryId;
@@ -151,27 +153,31 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         clientNode = this.ctx.clientNode();
 
-        clientMetaDataCache = clientNode ? new ConcurrentHashMap8<PortableMetaDataKey, BinaryType>() : null;
+        clientMetaDataCache = clientNode ? new ConcurrentHashMap8<Integer, BinaryTypeImpl>() : null;
     }
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         if (marsh instanceof PortableMarshaller) {
             PortableMetaDataHandler metaHnd = new PortableMetaDataHandler() {
-                @Override public void addMeta(int typeId, BinaryType newMeta)
-                    throws BinaryObjectException {
+                @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException {
+                    assert newMeta != null;
+                    assert newMeta instanceof BinaryTypeImpl;
+
+                    BinaryMetaDataImpl newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
+
                     if (metaDataCache == null) {
-                        BinaryType oldMeta = metaBuf.get(typeId);
+                        BinaryMetaDataImpl oldMeta = metaBuf.get(typeId);
 
-                        if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) {
+                        if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) {
                             synchronized (this) {
                                 Map<String, Integer> fields = new HashMap<>();
 
-                                if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                                    newMeta = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
-                                        newMeta.affinityKeyFieldName());
+                                if (checkMeta(typeId, oldMeta, newMeta0, fields)) {
+                                    newMeta0 = new BinaryMetaDataImpl(typeId, newMeta0.typeName(), fields,
+                                        newMeta0.affinityKeyFieldName());
 
-                                    metaBuf.put(typeId, newMeta);
+                                    metaBuf.put(typeId, newMeta0);
                                 }
                                 else
                                     return;
@@ -186,7 +192,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                             return;
                     }
 
-                    CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta);
+                    CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(portableCtx));
                 }
 
                 @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
@@ -232,7 +238,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
                 GridCacheQueryManager qryMgr = metaDataCache.context().queries();
 
-                CacheQuery<Map.Entry<PortableMetaDataKey, BinaryType>> qry =
+                CacheQuery<Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl>> qry =
                     qryMgr.createScanQuery(new MetaDataPredicate(), null, false);
 
                 qry.keepAll(false);
@@ -240,9 +246,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
 
                 try {
-                    CacheQueryFuture<Map.Entry<PortableMetaDataKey, BinaryType>> fut = qry.execute();
+                    CacheQueryFuture<Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl>> fut = qry.execute();
 
-                    Map.Entry<PortableMetaDataKey, BinaryType> next;
+                    Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl> next;
 
                     while ((next = fut.next()) != null) {
                         assert next.getKey() != null : next;
@@ -270,8 +276,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         startLatch.countDown();
 
-        for (Map.Entry<Integer, BinaryType> e : metaBuf.entrySet())
-            addMeta(e.getKey(), e.getValue());
+        for (Map.Entry<Integer, BinaryMetaDataImpl> e : metaBuf.entrySet())
+            addMeta(e.getKey(), e.getValue().wrap(portableCtx));
 
         metaBuf.clear();
     }
@@ -288,20 +294,24 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
      * @param key Metadata key.
      * @param newMeta Metadata.
      */
-    private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryType newMeta) {
-        clientMetaDataCache.compute(key,
-            new ConcurrentHashMap8.BiFun<PortableMetaDataKey, BinaryType, BinaryType>() {
-                @Override public BinaryType apply(PortableMetaDataKey key, BinaryType oldMeta) {
-                    BinaryType res;
+    private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryMetaDataImpl newMeta) {
+        int key0 = key.typeId();
+
+        clientMetaDataCache.compute(key0,
+            new ConcurrentHashMap8.BiFun<Integer, BinaryTypeImpl, BinaryTypeImpl>() {
+                @Override public BinaryTypeImpl apply(Integer key, BinaryTypeImpl oldMeta) {
+                    BinaryMetaDataImpl res;
+
+                    BinaryMetaDataImpl oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
 
                     try {
-                        res = checkMeta(key.typeId(), oldMeta, newMeta, null) ? newMeta : oldMeta;
+                        res = checkMeta(key, oldMeta0, newMeta, null) ? newMeta : oldMeta0;
                     }
                     catch (BinaryObjectException e) {
-                        res = oldMeta;
+                        res = oldMeta0;
                     }
 
-                    return res;
+                    return res != null ? res.wrap(portableCtx) : null;
                 }
             }
         );
@@ -446,14 +456,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @Override public void addMeta(final int typeId, final BinaryType newMeta) throws BinaryObjectException {
         assert newMeta != null;
+        assert newMeta instanceof BinaryTypeImpl;
+
+        BinaryMetaDataImpl newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
 
         final PortableMetaDataKey key = new PortableMetaDataKey(typeId);
 
         try {
-            BinaryType oldMeta = metaDataCache.localPeek(key);
+            BinaryMetaDataImpl oldMeta = metaDataCache.localPeek(key);
 
-            if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) {
-                BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta));
+            if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) {
+                BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta0));
 
                 if (err != null)
                     throw err;
@@ -468,9 +481,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     @Nullable @Override public BinaryType metadata(final int typeId) throws BinaryObjectException {
         try {
             if (clientNode)
-                return clientMetaDataCache.get(new PortableMetaDataKey(typeId));
+                return clientMetaDataCache.get(typeId);
+            else {
+                BinaryMetaDataImpl meta = metaDataCache.localPeek(new PortableMetaDataKey(typeId));
 
-            return metaDataCache.localPeek(new PortableMetaDataKey(typeId));
+                return meta != null ? meta.wrap(portableCtx) : null;
+            }
         }
         catch (CacheException e) {
             throw new BinaryObjectException(e);
@@ -486,12 +502,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             for (Integer typeId : typeIds)
                 keys.add(new PortableMetaDataKey(typeId));
 
-            Map<PortableMetaDataKey, BinaryType> meta = metaDataCache.getAll(keys);
+            Map<PortableMetaDataKey, BinaryMetaDataImpl> meta = metaDataCache.getAll(keys);
 
             Map<Integer, BinaryType> res = U.newHashMap(meta.size());
 
-            for (Map.Entry<PortableMetaDataKey, BinaryType> e : meta.entrySet())
-                res.put(e.getKey().typeId(), e.getValue());
+            for (Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl> e : meta.entrySet())
+                res.put(e.getKey().typeId(), e.getValue().wrap(portableCtx));
 
             return res;
         }
@@ -504,17 +520,21 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     @SuppressWarnings("unchecked")
     @Override public Collection<BinaryType> metadata() throws BinaryObjectException {
         if (clientNode)
-            return new ArrayList<>(clientMetaDataCache.values());
-
-        return F.viewReadOnly(metaDataCache.entrySetx(metaPred),
-            new C1<Cache.Entry<PortableMetaDataKey, BinaryType>, BinaryType>() {
-                private static final long serialVersionUID = 0L;
-
-                @Override public BinaryType apply(
-                    Cache.Entry<PortableMetaDataKey, BinaryType> e) {
-                    return e.getValue();
+            return F.viewReadOnly(clientMetaDataCache.values(), new IgniteClosure<BinaryTypeImpl, BinaryType>() {
+                @Override public BinaryType apply(BinaryTypeImpl meta) {
+                    return meta;
                 }
             });
+        else {
+            return F.viewReadOnly(metaDataCache.entrySetx(metaPred),
+                new C1<Cache.Entry<PortableMetaDataKey, BinaryMetaDataImpl>, BinaryType>() {
+                    private static final long serialVersionUID = 0L;
+
+                    @Override public BinaryType apply(Cache.Entry<PortableMetaDataKey, BinaryMetaDataImpl> e) {
+                        return e.getValue().wrap(portableCtx);
+                    }
+                });
+        }
     }
 
     /** {@inheritDoc} */
@@ -714,12 +734,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
      * @return Whether meta is changed.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    private static boolean checkMeta(int typeId, @Nullable BinaryType oldMeta,
-        BinaryType newMeta, @Nullable Map<String, Integer> fields) throws BinaryObjectException {
+    private static boolean checkMeta(int typeId, @Nullable BinaryMetaDataImpl oldMeta,
+        BinaryMetaDataImpl newMeta, @Nullable Map<String, Integer> fields) throws BinaryObjectException {
         assert newMeta != null;
 
-        Map<String, Integer> oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fields0() : null;
-        Map<String, Integer> newFields = ((BinaryMetaDataImpl)newMeta).fields0();
+        Map<String, Integer> oldFields = oldMeta != null ? oldMeta.fieldsMap() : null;
+        Map<String, Integer> newFields = newMeta.fieldsMap();
 
         boolean changed = false;
 
@@ -779,7 +799,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /**
      */
     private static class MetaDataProcessor implements
-        EntryProcessor<PortableMetaDataKey, BinaryType, BinaryObjectException>, Externalizable {
+        EntryProcessor<PortableMetaDataKey, BinaryMetaDataImpl, BinaryObjectException>, Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -787,7 +807,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         private int typeId;
 
         /** */
-        private BinaryType newMeta;
+        private BinaryMetaDataImpl newMeta;
 
         /**
          * For {@link Externalizable}.
@@ -800,7 +820,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
          * @param typeId Type ID.
          * @param newMeta New metadata.
          */
-        private MetaDataProcessor(int typeId, BinaryType newMeta) {
+        private MetaDataProcessor(int typeId, BinaryMetaDataImpl newMeta) {
             assert newMeta != null;
 
             this.typeId = typeId;
@@ -809,15 +829,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         /** {@inheritDoc} */
         @Override public BinaryObjectException process(
-            MutableEntry<PortableMetaDataKey, BinaryType> entry,
+            MutableEntry<PortableMetaDataKey, BinaryMetaDataImpl> entry,
             Object... args) {
             try {
-                BinaryType oldMeta = entry.getValue();
+                BinaryMetaDataImpl oldMeta = entry.getValue();
 
                 Map<String, Integer> fields = new HashMap<>();
 
                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                    BinaryType res = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
+                    BinaryMetaDataImpl res = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
                         newMeta.affinityKeyFieldName());
 
                     entry.setValue(res);
@@ -841,7 +861,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             typeId = in.readInt();
-            newMeta = (BinaryType)in.readObject();
+            newMeta = (BinaryMetaDataImpl)in.readObject();
         }
 
         /** {@inheritDoc} */
@@ -853,17 +873,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /**
      *
      */
-    class MetaDataEntryListener implements CacheEntryUpdatedListener<PortableMetaDataKey, BinaryType> {
+    class MetaDataEntryListener implements CacheEntryUpdatedListener<PortableMetaDataKey, BinaryMetaDataImpl> {
         /** {@inheritDoc} */
         @Override public void onUpdated(
-            Iterable<CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryType>> evts)
+            Iterable<CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryMetaDataImpl>> evts)
             throws CacheEntryListenerException {
-            for (CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryType> evt : evts) {
+            for (CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryMetaDataImpl> evt : evts) {
                 assert evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED : evt;
 
                 PortableMetaDataKey key = evt.getKey();
 
-                final BinaryType newMeta = evt.getValue();
+                final BinaryMetaDataImpl newMeta = evt.getValue();
 
                 assert newMeta != null : evt;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/efe211b0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index af99815..fdc0e0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -390,7 +390,7 @@ public class PlatformContextImpl implements PlatformContext {
         else {
             writer.writeBoolean(true);
 
-            Map<String, Integer> fields = ((BinaryMetaDataImpl)meta).fields0();
+            Map<String, Integer> fields = ((BinaryMetaDataImpl)meta).fieldsMap();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());