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:41 UTC

[1/9] ignite git commit: IGNITE-1847: Fixed rude bugs in BinaryMetaDataImpl.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1847 [created] b2fd7bfd4


IGNITE-1847: Fixed rude bugs in BinaryMetaDataImpl.


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

Branch: refs/heads/ignite-1847
Commit: 52b57c7bf46539f3cc626a3ee40c0ec0d688424a
Parents: 93b71c7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Nov 9 15:50:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Nov 9 15:50:08 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryMetaDataImpl.java   | 30 +-------------------
 .../internal/portable/PortableContext.java      |  3 --
 .../CacheObjectBinaryProcessorImpl.java         |  4 +--
 3 files changed, 3 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/52b57c7b/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 18f538b..7f2bdb9 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
@@ -39,7 +39,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Portable meta data implementation.
  */
-public class BinaryMetaDataImpl implements BinaryType, Binarylizable, Externalizable {
+public class BinaryMetaDataImpl implements BinaryType, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -51,9 +51,6 @@ public class BinaryMetaDataImpl implements BinaryType, Binarylizable, Externaliz
     private Map<String, String> fields;
 
     /** */
-    private volatile Map<Integer, String> fldIdToName;
-
-    /** */
     private String affKeyFieldName;
 
     /**
@@ -104,13 +101,6 @@ public class BinaryMetaDataImpl implements BinaryType, Binarylizable, Externaliz
         return affKeyFieldName;
     }
 
-    /**
-     * @return Fields meta data.
-     */
-    public Map<String, String> fieldsMeta() {
-        return fields != null ? fields : Collections.<String, String>emptyMap();
-    }
-
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeString(out, typeName);
@@ -126,24 +116,6 @@ public class BinaryMetaDataImpl implements BinaryType, Binarylizable, Externaliz
     }
 
     /** {@inheritDoc} */
-    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
-        BinaryRawWriter raw = writer.rawWriter();
-
-        raw.writeString(typeName);
-        raw.writeString(affKeyFieldName);
-        raw.writeMap(fields);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
-        BinaryRawReader raw = reader.rawReader();
-
-        typeName = raw.readString();
-        affKeyFieldName = raw.readString();
-        fields = raw.readMap();
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(BinaryMetaDataImpl.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/52b57c7b/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 54a180b..3c5633f 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
@@ -237,9 +237,6 @@ public class PortableContext implements Externalizable {
         registerPredefinedType(T2.class, 62);
 
         // IDs range [200..1000] is used by Ignite internal APIs.
-
-        registerPredefinedType(BinaryObjectImpl.class, 200);
-        registerPredefinedType(BinaryMetaDataImpl.class, 201);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/52b57c7b/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 2b3aa09..7c9b926 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
@@ -843,8 +843,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         BinaryType newMeta, @Nullable Map<String, String> fields) throws BinaryObjectException {
         assert newMeta != null;
 
-        Map<String, String> oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fieldsMeta() : null;
-        Map<String, String> newFields = ((BinaryMetaDataImpl)newMeta).fieldsMeta();
+        Map<String, String> oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fields0() : null;
+        Map<String, String> newFields = ((BinaryMetaDataImpl)newMeta).fields0();
 
         boolean changed = false;
 


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

Posted by vo...@apache.org.
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());


[2/9] ignite git commit: IGNITE-1847: Added typeId.

Posted by vo...@apache.org.
IGNITE-1847: Added typeId.


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

Branch: refs/heads/ignite-1847
Commit: 267046b058e6ed7ee62f00b93878176618d05b65
Parents: 52b57c7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Nov 9 16:13:37 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Nov 9 16:13:37 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryMetaDataImpl.java   | 30 +++++++++++++-------
 .../internal/portable/PortableContext.java      |  4 +--
 .../CacheObjectBinaryProcessorImpl.java         | 10 +++----
 3 files changed, 26 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/267046b0/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 7f2bdb9..68bc444 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,7 @@ 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.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryRawReader;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -43,14 +37,17 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** */
+    /** Type ID. */
+    private int typeId;
+
+    /** Type name. */
     private String typeName;
 
-    /** */
+    /** Recorded object fields. */
     @GridToStringInclude
     private Map<String, String> fields;
 
-    /** */
+    /** Affinity key field name. */
     private String affKeyFieldName;
 
     /**
@@ -61,19 +58,30 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
     }
 
     /**
+     * Constructor.
+     *
+     * @param typeId Type ID.
      * @param typeName Type name.
      * @param fields Fields map.
      * @param affKeyFieldName Affinity key field name.
      */
-    public BinaryMetaDataImpl(String typeName, @Nullable Map<String, String> fields,
+    public BinaryMetaDataImpl(int typeId, String typeName, @Nullable Map<String, String> fields,
         @Nullable String affKeyFieldName) {
         assert typeName != null;
 
+        this.typeId = typeId;
         this.typeName = typeName;
         this.fields = fields;
         this.affKeyFieldName = affKeyFieldName;
     }
 
+    /**
+     * @return Type ID.
+     */
+    public int typeId() {
+        return typeId;
+    }
+
     /** {@inheritDoc} */
     @Override public String typeName() {
         return typeName;
@@ -103,6 +111,7 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(typeId);
         U.writeString(out, typeName);
         U.writeMap(out, fields);
         U.writeString(out, affKeyFieldName);
@@ -110,6 +119,7 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        typeId = in.readInt();
         typeName = U.readString(in);
         fields = U.readMap(in);
         affKeyFieldName = U.readString(in);

http://git-wip-us.apache.org/repos/asf/ignite/blob/267046b0/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 3c5633f..84cc748 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
@@ -776,7 +776,7 @@ public class PortableContext implements Externalizable {
             descByCls.put(cls, desc);
         }
 
-        metaHnd.addMeta(id, new BinaryMetaDataImpl(typeName, fieldsMeta, affKeyFieldName));
+        metaHnd.addMeta(id, new BinaryMetaDataImpl(id, typeName, fieldsMeta, affKeyFieldName));
     }
 
     /**
@@ -816,7 +816,7 @@ public class PortableContext implements Externalizable {
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
     public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws BinaryObjectException {
-        updateMetaData(typeId, new BinaryMetaDataImpl(typeName, fields, null));
+        updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fields, null));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/267046b0/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 7c9b926..7e9a554 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
@@ -292,8 +292,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                                 Map<String, String> fields = new HashMap<>();
 
                                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                                    newMeta = new BinaryMetaDataImpl(newMeta.typeName(),
-                                        fields,
+                                    newMeta = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
                                         newMeta.affinityKeyFieldName());
 
                                     metaBuf.put(typeId, newMeta);
@@ -564,8 +563,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
         Map<String, Integer> fieldTypeIds) throws BinaryObjectException {
-        portableCtx.updateMetaData(typeId,
-            new BinaryMetaDataImpl(typeName, fieldTypeNames(fieldTypeIds), affKeyFieldName));
+        portableCtx.updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fieldTypeNames(fieldTypeIds),
+            affKeyFieldName));
     }
 
     /** {@inheritDoc} */
@@ -942,8 +941,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 Map<String, String> fields = new HashMap<>();
 
                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                    BinaryType res = new BinaryMetaDataImpl(newMeta.typeName(),
-                        fields,
+                    BinaryType res = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
                         newMeta.affinityKeyFieldName());
 
                     entry.setValue(res);


[3/9] ignite git commit: IGNITE-1847: Refactored field type names.

Posted by vo...@apache.org.
IGNITE-1847: Refactored field type names.


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

Branch: refs/heads/ignite-1847
Commit: b4c787b7ec62862372091815cf08f65791a3956f
Parents: 267046b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Nov 9 16:26:22 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Nov 9 16:26:22 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/portable/PortableUtils.java |  88 +++++++++
 .../builder/BinaryObjectBuilderImpl.java        |  19 +-
 .../portable/builder/PortableValueWithType.java |   4 +-
 .../CacheObjectBinaryProcessorImpl.java         | 189 ++++---------------
 .../platform/PlatformContextImpl.java           |   3 +-
 5 files changed, 134 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b4c787b7/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index fe97e7e..a22f1aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -49,6 +49,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
@@ -69,6 +70,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENT
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROTO_VER;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
@@ -116,6 +118,92 @@ public class PortableUtils {
     /** Offset which fits into 4 bytes. */
     public static final int OFFSET_4 = 4;
 
+    /** Field type names. */
+    private static final String[] FIELD_TYPE_NAMES;
+
+    static {
+        FIELD_TYPE_NAMES = new String[104];
+
+        FIELD_TYPE_NAMES[BYTE] = "byte";
+        FIELD_TYPE_NAMES[SHORT] = "short";
+        FIELD_TYPE_NAMES[INT] = "int";
+        FIELD_TYPE_NAMES[LONG] = "long";
+        FIELD_TYPE_NAMES[BOOLEAN] = "boolean";
+        FIELD_TYPE_NAMES[FLOAT] = "float";
+        FIELD_TYPE_NAMES[DOUBLE] = "double";
+        FIELD_TYPE_NAMES[CHAR] = "char";
+        FIELD_TYPE_NAMES[UUID] = "UUID";
+        FIELD_TYPE_NAMES[DECIMAL] = "decimal";
+        FIELD_TYPE_NAMES[STRING] = "String";
+        FIELD_TYPE_NAMES[DATE] = "Date";
+        FIELD_TYPE_NAMES[TIMESTAMP] = "Timestamp";
+        FIELD_TYPE_NAMES[ENUM] = "Enum";
+        FIELD_TYPE_NAMES[OBJ] = "Object";
+        FIELD_TYPE_NAMES[PORTABLE_OBJ] = "Object";
+        FIELD_TYPE_NAMES[COL] = "Collection";
+        FIELD_TYPE_NAMES[MAP] = "Map";
+        FIELD_TYPE_NAMES[MAP_ENTRY] = "Entry";
+        FIELD_TYPE_NAMES[CLASS] = "Class";
+        FIELD_TYPE_NAMES[BYTE_ARR] = "byte[]";
+        FIELD_TYPE_NAMES[SHORT_ARR] = "short[]";
+        FIELD_TYPE_NAMES[INT_ARR] = "int[]";
+        FIELD_TYPE_NAMES[LONG_ARR] = "long[]";
+        FIELD_TYPE_NAMES[BOOLEAN_ARR] = "boolean[]";
+        FIELD_TYPE_NAMES[FLOAT_ARR] = "float[]";
+        FIELD_TYPE_NAMES[DOUBLE_ARR] = "double[]";
+        FIELD_TYPE_NAMES[CHAR_ARR] = "char[]";
+        FIELD_TYPE_NAMES[UUID_ARR] = "UUID[]";
+        FIELD_TYPE_NAMES[DECIMAL_ARR] = "decimal[]";
+        FIELD_TYPE_NAMES[STRING_ARR] = "String[]";
+        FIELD_TYPE_NAMES[DATE_ARR] = "Date[]";
+        FIELD_TYPE_NAMES[TIMESTAMP_ARR] = "Timestamp[]";
+        FIELD_TYPE_NAMES[OBJ_ARR] = "Object[]";
+        FIELD_TYPE_NAMES[ENUM_ARR] = "Enum[]";
+    }
+
+    /**
+     * @param typeName Field type name.
+     * @return Field type ID;
+     */
+    @SuppressWarnings("StringEquality")
+    public static int fieldTypeId(String typeName) {
+        for (int i = 0; i < FIELD_TYPE_NAMES.length; i++) {
+            String typeName0 = FIELD_TYPE_NAMES[i];
+
+            if (typeName.equals(typeName0))
+                return i;
+        }
+
+        throw new IllegalArgumentException("Invalid metadata type name: " + typeName);
+    }
+
+    /**
+     * @param typeId Field type ID.
+     * @return Field type name.
+     */
+    public static String fieldTypeName(int typeId) {
+        assert typeId >= 0 && typeId < FIELD_TYPE_NAMES.length : typeId;
+
+        String typeName = FIELD_TYPE_NAMES[typeId];
+
+        assert typeName != null : typeId;
+
+        return typeName;
+    }
+
+    /**
+     * @param typeIds Field type IDs.
+     * @return Field type names.
+     */
+    public static Map<String, String> fieldTypeNames(Map<String, Integer> typeIds) {
+        Map<String, String> names = U.newHashMap(typeIds.size());
+
+        for (Map.Entry<String, Integer> e : typeIds.entrySet())
+            names.put(e.getKey(), fieldTypeName(e.getValue()));
+
+        return names;
+    }
+
     /**
      * Write flags.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4c787b7/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
index 777d30b..85f2b68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
@@ -17,15 +17,10 @@
 
 package org.apache.ignite.internal.portable.builder;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
@@ -33,13 +28,18 @@ import org.apache.ignite.internal.portable.BinaryWriterExImpl;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableUtils;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLAGS_POS;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
@@ -312,7 +312,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                     else {
                         byte type = PortableUtils.typeByClass(val.getClass());
 
-                        newFldTypeName = CacheObjectBinaryProcessorImpl.fieldTypeName(type);
+                        newFldTypeName = PortableUtils.fieldTypeName(type);
                     }
 
                     if (oldFldTypeName == null) {
@@ -324,8 +324,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                         newFldsMetadata.put(name, newFldTypeName);
                     }
                     else {
-                        String objTypeName =
-                            CacheObjectBinaryProcessorImpl.FIELD_TYPE_NAMES[GridPortableMarshaller.OBJ];
+                        String objTypeName = PortableUtils.fieldTypeName(GridPortableMarshaller.OBJ);
 
                         if (!objTypeName.equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
                             throw new BinaryObjectException(

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4c787b7/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
index 5ea8e62..66d351b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.portable.builder;
 
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
@@ -50,7 +50,7 @@ class PortableValueWithType implements PortableLazyValue {
 
     /** {@inheritDoc} */
     public String typeName() {
-        return CacheObjectBinaryProcessorImpl.fieldTypeName(type);
+        return PortableUtils.fieldTypeName(type);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4c787b7/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 7e9a554..49f4fd8 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
@@ -17,41 +17,25 @@
 
 package org.apache.ignite.internal.processors.cache.portable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import javax.cache.Cache;
-import javax.cache.CacheException;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import javax.cache.event.CacheEntryUpdatedListener;
-import javax.cache.event.EventType;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterTopologyException;
-import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
-import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.PortableMetaDataHandler;
+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.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.internal.portable.PortableMetaDataHandler;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
@@ -84,49 +68,30 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
 
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_ARR;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import javax.cache.event.EventType;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
 
 /**
  * Portable processor implementation.
@@ -134,9 +99,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_AR
 public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorImpl implements
     CacheObjectBinaryProcessor {
     /** */
-    public static final String[] FIELD_TYPE_NAMES;
-
-    /** */
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
     /** */
@@ -180,92 +142,6 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private UUID metaCacheQryId;
 
     /**
-     *
-     */
-    static {
-        FIELD_TYPE_NAMES = new String[104];
-
-        FIELD_TYPE_NAMES[BYTE] = "byte";
-        FIELD_TYPE_NAMES[SHORT] = "short";
-        FIELD_TYPE_NAMES[INT] = "int";
-        FIELD_TYPE_NAMES[LONG] = "long";
-        FIELD_TYPE_NAMES[BOOLEAN] = "boolean";
-        FIELD_TYPE_NAMES[FLOAT] = "float";
-        FIELD_TYPE_NAMES[DOUBLE] = "double";
-        FIELD_TYPE_NAMES[CHAR] = "char";
-        FIELD_TYPE_NAMES[UUID] = "UUID";
-        FIELD_TYPE_NAMES[DECIMAL] = "decimal";
-        FIELD_TYPE_NAMES[STRING] = "String";
-        FIELD_TYPE_NAMES[DATE] = "Date";
-        FIELD_TYPE_NAMES[TIMESTAMP] = "Timestamp";
-        FIELD_TYPE_NAMES[ENUM] = "Enum";
-        FIELD_TYPE_NAMES[OBJ] = "Object";
-        FIELD_TYPE_NAMES[PORTABLE_OBJ] = "Object";
-        FIELD_TYPE_NAMES[COL] = "Collection";
-        FIELD_TYPE_NAMES[MAP] = "Map";
-        FIELD_TYPE_NAMES[MAP_ENTRY] = "Entry";
-        FIELD_TYPE_NAMES[CLASS] = "Class";
-        FIELD_TYPE_NAMES[BYTE_ARR] = "byte[]";
-        FIELD_TYPE_NAMES[SHORT_ARR] = "short[]";
-        FIELD_TYPE_NAMES[INT_ARR] = "int[]";
-        FIELD_TYPE_NAMES[LONG_ARR] = "long[]";
-        FIELD_TYPE_NAMES[BOOLEAN_ARR] = "boolean[]";
-        FIELD_TYPE_NAMES[FLOAT_ARR] = "float[]";
-        FIELD_TYPE_NAMES[DOUBLE_ARR] = "double[]";
-        FIELD_TYPE_NAMES[CHAR_ARR] = "char[]";
-        FIELD_TYPE_NAMES[UUID_ARR] = "UUID[]";
-        FIELD_TYPE_NAMES[DECIMAL_ARR] = "decimal[]";
-        FIELD_TYPE_NAMES[STRING_ARR] = "String[]";
-        FIELD_TYPE_NAMES[DATE_ARR] = "Date[]";
-        FIELD_TYPE_NAMES[TIMESTAMP_ARR] = "Timestamp[]";
-        FIELD_TYPE_NAMES[OBJ_ARR] = "Object[]";
-        FIELD_TYPE_NAMES[ENUM_ARR] = "Enum[]";
-    }
-
-    /**
-     * @param typeName Field type name.
-     * @return Field type ID;
-     */
-    @SuppressWarnings("StringEquality")
-    public static int fieldTypeId(String typeName) {
-        for (int i = 0; i < FIELD_TYPE_NAMES.length; i++) {
-            String typeName0 = FIELD_TYPE_NAMES[i];
-
-            if (typeName.equals(typeName0))
-                return i;
-        }
-
-        throw new IllegalArgumentException("Invalid metadata type name: " + typeName);
-    }
-
-    /**
-     * @param typeId Field type ID.
-     * @return Field type name.
-     */
-    public static String fieldTypeName(int typeId) {
-        assert typeId >= 0 && typeId < FIELD_TYPE_NAMES.length : typeId;
-
-        String typeName = FIELD_TYPE_NAMES[typeId];
-
-        assert typeName != null : typeId;
-
-        return typeName;
-    }
-
-    /**
-     * @param typeIds Field type IDs.
-     * @return Field type names.
-     */
-    public static Map<String, String> fieldTypeNames(Map<String, Integer> typeIds) {
-        Map<String, String> names = U.newHashMap(typeIds.size());
-
-        for (Map.Entry<String, Integer> e : typeIds.entrySet())
-            names.put(e.getKey(), fieldTypeName(e.getValue()));
-
-        return names;
-    }
-
-    /**
      * @param ctx Kernal context.
      */
     public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) {
@@ -479,6 +355,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public Object marshalToPortable(@Nullable Object obj) throws BinaryObjectException {
         if (obj == null)
             return null;
@@ -563,8 +440,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
         Map<String, Integer> fieldTypeIds) throws BinaryObjectException {
-        portableCtx.updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fieldTypeNames(fieldTypeIds),
-            affKeyFieldName));
+        portableCtx.updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName,
+            PortableUtils.fieldTypeNames(fieldTypeIds), affKeyFieldName));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4c787b7/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 c513600..6dc78e9 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
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.BinaryMetaDataImpl;
 import org.apache.ignite.internal.portable.BinaryRawReaderEx;
 import org.apache.ignite.internal.portable.BinaryRawWriterEx;
+import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl;
@@ -396,7 +397,7 @@ public class PlatformContextImpl implements PlatformContext {
             Map<String, Integer> fields = U.newHashMap(metaFields.size());
 
             for (Map.Entry<String, String> metaField : metaFields.entrySet())
-                fields.put(metaField.getKey(), CacheObjectBinaryProcessorImpl.fieldTypeId(metaField.getValue()));
+                fields.put(metaField.getKey(), PortableUtils.fieldTypeId(metaField.getValue()));
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());


[7/9] ignite git commit: IGNITE-1847: WIP.

Posted by vo...@apache.org.
IGNITE-1847: WIP.


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

Branch: refs/heads/ignite-1847
Commit: a6e214820183207edf3dcbc4d4c08f957e0ff16d
Parents: efe211b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 10 13:34:51 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 10 13:34:51 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/binary/BinaryType.java | 11 ++++++++++-
 .../ignite/internal/portable/BinaryObjectEx.java  |  4 ++--
 .../ignite/internal/portable/BinaryTypeImpl.java  | 18 ++++++------------
 ...GridBinaryObjectBuilderAdditionalSelfTest.java |  2 +-
 .../portable/GridBinaryObjectBuilderSelfTest.java |  4 ++--
 .../portable/GridPortableMetaDataSelfTest.java    | 12 ++++++------
 ...ientNodeBinaryObjectMetadataMultinodeTest.java |  4 ++--
 ...idCacheClientNodeBinaryObjectMetadataTest.java |  5 +----
 .../platform/PlatformComputePortableArgTask.java  |  2 +-
 9 files changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
index d149fd4..52bb212 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
@@ -38,7 +38,7 @@ public interface BinaryType {
      *
      * @return Collection of all field names for this binary type.
      */
-    public Collection<String> fields();
+    public Collection<String> fieldNames();
 
     /**
      * Gets name of the field type for a given field.
@@ -49,6 +49,15 @@ public interface BinaryType {
     public String fieldTypeName(String fieldName);
 
     /**
+     * Get {@link BinaryField} for the given field name. Later this field can be used for fast field's value
+     * retrieval from concrete {@link BinaryObject}.
+     *
+     * @param fieldName Field name.
+     * @return Binary field.
+     */
+    public BinaryField field(String fieldName);
+
+    /**
      * Binary objects can optionally specify custom key-affinity mapping in the
      * configuration. This method returns the name of the field which should be
      * used for the key-affinity mapping.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
index 50b9d42..b3512ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
@@ -176,10 +176,10 @@ public abstract class BinaryObjectEx implements BinaryObject {
 
         SB buf = new SB(meta.typeName());
 
-        if (meta.fields() != null) {
+        if (meta.fieldNames() != null) {
             buf.a(" [hash=").a(idHash);
 
-            for (String name : meta.fields()) {
+            for (String name : meta.fieldNames()) {
                 Object val = field(ctx, name);
 
                 buf.a(", ").a(name).a('=');

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/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
index d1af2fe..9effdce 100644
--- 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
@@ -48,7 +48,7 @@ public class BinaryTypeImpl implements BinaryType {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<String> fields() {
+    @Override public Collection<String> fieldNames() {
         return meta.fields();
     }
 
@@ -58,7 +58,11 @@ public class BinaryTypeImpl implements BinaryType {
     }
 
     /** {@inheritDoc} */
-    @Override public String affinityKeyFieldName() {
+    @Override public BinaryFieldImpl field(String fieldName) {
+        return ctx.createField(meta.typeId(), fieldName);
+    }
+
+    public String affinityKeyFieldName() {
         return meta.affinityKeyFieldName();
     }
 
@@ -68,14 +72,4 @@ public class BinaryTypeImpl implements BinaryType {
     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/a6e21482/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
index e62d12e..11b54ae 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
@@ -998,7 +998,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac
 
         BinaryType metadata = portables().metadata(c.getClass());
 
-        assertTrue(metadata.fields().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
+        assertTrue(metadata.fieldNames().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
             "colField", "mapField", "enumField", "enumArrField")));
 
         assertEquals("int", metadata.fieldTypeName("intField"));

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
index c697510..3c69440 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
@@ -789,7 +789,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         assertEquals("MetaTest", meta.typeName());
 
-        Collection<String> fields = meta.fields();
+        Collection<String> fields = meta.fieldNames();
 
         assertEquals(2, fields.size());
 
@@ -812,7 +812,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         assertEquals("MetaTest", meta.typeName());
 
-        fields = meta.fields();
+        fields = meta.fieldNames();
 
         assertEquals(3, fields.size());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
index c0e2563..2a367a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
@@ -93,7 +93,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
             switch (meta.typeName()) {
                 case "TestObject1":
-                    fields = meta.fields();
+                    fields = meta.fieldNames();
 
                     assertEquals(7, fields.size());
 
@@ -116,7 +116,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
                     break;
 
                 case "TestObject2":
-                    fields = meta.fields();
+                    fields = meta.fieldNames();
 
                     assertEquals(7, fields.size());
 
@@ -165,7 +165,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
         assertEquals("TestObject1", meta.typeName());
 
-        Collection<String> fields = meta.fields();
+        Collection<String> fields = meta.fieldNames();
 
         assertEquals(7, fields.size());
 
@@ -198,7 +198,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
         assertEquals("TestObject2", meta.typeName());
 
-        Collection<String> fields = meta.fields();
+        Collection<String> fields = meta.fieldNames();
 
         assertEquals(7, fields.size());
 
@@ -235,7 +235,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
         assertEquals("TestObject2", meta.typeName());
 
-        Collection<String> fields = meta.fields();
+        Collection<String> fields = meta.fieldNames();
 
         assertEquals(9, fields.size());
 
@@ -284,7 +284,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
         assertEquals("TestObject1", meta.typeName());
 
-        Collection<String> fields = meta.fields();
+        Collection<String> fields = meta.fieldNames();
 
         assertEquals(7, fields.size());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
index a33eb7b..d19c1ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
@@ -169,7 +169,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
 
                 assertNull(meta.affinityKeyFieldName());
 
-                assertEquals(10, meta.fields().size());
+                assertEquals(10, meta.fieldNames().size());
             }
 
             assertEquals(allTypes.size(), names.size());
@@ -255,7 +255,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
 
                 assertNull(meta.affinityKeyFieldName());
 
-                assertEquals(1, meta.fields().size());
+                assertEquals(1, meta.fieldNames().size());
             }
 
             assertEquals(1000, names.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
index b20adb8..10c06a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.portable;
 import java.util.Arrays;
 import java.util.Collection;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.CacheMode;
@@ -29,9 +28,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 
@@ -127,7 +124,7 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra
             for (BinaryType m2 : meta1) {
                 if (m1.typeName().equals(m2.typeName())) {
                     assertEquals(m1.affinityKeyFieldName(), m2.affinityKeyFieldName());
-                    assertEquals(m1.fields(), m2.fields());
+                    assertEquals(m1.fieldNames(), m2.fieldNames());
 
                     found = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e21482/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
index 1e3c499..cae7a6f 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
@@ -94,7 +94,7 @@ public class PlatformComputePortableArgTask extends ComputeTaskAdapter<Object, I
             if (meta == null)
                 throw new IgniteException("Metadata doesn't exist.");
 
-            if (meta.fields() == null || !meta.fields().contains("Field"))
+            if (meta.fieldNames() == null || !meta.fieldNames().contains("Field"))
                 throw new IgniteException("Field metadata doesn't exist.");
 
             if (!F.eq("int", meta.fieldTypeName("Field")))


[5/9] ignite git commit: IGNITE-1847: Metadata fields: (string, string) -> (string, int)

Posted by vo...@apache.org.
IGNITE-1847: Metadata fields: (string,string) -> (string, int)


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

Branch: refs/heads/ignite-1847
Commit: cf108619ba01489275aaa845cad327c169b653d2
Parents: fde2b3e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Nov 9 17:34:54 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Nov 9 17:34:54 2015 +0300

----------------------------------------------------------------------
 .../portable/BinaryMetaDataCollector.java       | 95 +++++++++-----------
 .../internal/portable/BinaryMetaDataImpl.java   | 12 +--
 .../portable/PortableClassDescriptor.java       | 26 +++---
 .../internal/portable/PortableContext.java      |  5 +-
 .../builder/BinaryObjectBuilderImpl.java        | 15 ++--
 .../portable/builder/PortableValueWithType.java |  9 +-
 .../CacheObjectBinaryProcessorImpl.java         | 25 +++---
 .../platform/PlatformContextImpl.java           |  9 +-
 8 files changed, 91 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
index b053a55..b9ca926 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
@@ -37,7 +37,7 @@ import org.jetbrains.annotations.Nullable;
  */
 class BinaryMetaDataCollector implements BinaryWriter {
     /** */
-    private final Map<String, String> meta = new HashMap<>();
+    private final Map<String, Integer> meta = new HashMap<>();
 
     /** */
     private final String typeName;
@@ -52,169 +52,169 @@ class BinaryMetaDataCollector implements BinaryWriter {
     /**
      * @return Field meta data.
      */
-    Map<String, String> meta() {
+    Map<String, Integer> meta() {
         return meta;
     }
 
     /** {@inheritDoc} */
     @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException {
-        add(fieldName, byte.class);
+        add(fieldName, PortableClassDescriptor.Mode.BYTE);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(String fieldName, short val) throws BinaryObjectException {
-        add(fieldName, short.class);
+        add(fieldName, PortableClassDescriptor.Mode.SHORT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeInt(String fieldName, int val) throws BinaryObjectException {
-        add(fieldName, int.class);
+        add(fieldName, PortableClassDescriptor.Mode.INT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLong(String fieldName, long val) throws BinaryObjectException {
-        add(fieldName, long.class);
+        add(fieldName, PortableClassDescriptor.Mode.LONG);
     }
 
     /** {@inheritDoc} */
     @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException {
-        add(fieldName, float.class);
+        add(fieldName, PortableClassDescriptor.Mode.FLOAT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException {
-        add(fieldName, double.class);
+        add(fieldName, PortableClassDescriptor.Mode.DOUBLE);
     }
 
     /** {@inheritDoc} */
     @Override public void writeChar(String fieldName, char val) throws BinaryObjectException {
-        add(fieldName, char.class);
+        add(fieldName, PortableClassDescriptor.Mode.CHAR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException {
-        add(fieldName, boolean.class);
+        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DECIMAL.typeName());
+        add(fieldName, PortableClassDescriptor.Mode.DECIMAL);
     }
 
     /** {@inheritDoc} */
     @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException {
-        add(fieldName, String.class);
+        add(fieldName, PortableClassDescriptor.Mode.STRING);
     }
 
     /** {@inheritDoc} */
     @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException {
-        add(fieldName, UUID.class);
+        add(fieldName, PortableClassDescriptor.Mode.UUID);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException {
-        add(fieldName, Date.class);
+        add(fieldName, PortableClassDescriptor.Mode.DATE);
     }
 
     /** {@inheritDoc} */
     @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException {
-        add(fieldName, Timestamp.class);
+        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP);
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws BinaryObjectException {
-        add(fieldName, Enum.class);
+        add(fieldName, PortableClassDescriptor.Mode.ENUM);
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException {
-        add(fieldName, Enum[].class);
+        add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException {
-        add(fieldName, Object.class);
+        add(fieldName, PortableClassDescriptor.Mode.OBJECT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException {
-        add(fieldName, byte[].class);
+        add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException {
-        add(fieldName, short[].class);
+        add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException {
-        add(fieldName, int[].class);
+        add(fieldName, PortableClassDescriptor.Mode.INT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException {
-        add(fieldName, long[].class);
+        add(fieldName, PortableClassDescriptor.Mode.LONG_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException {
-        add(fieldName, float[].class);
+        add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException {
-        add(fieldName, double[].class);
+        add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException {
-        add(fieldName, char[].class);
+        add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException {
-        add(fieldName, boolean[].class);
+        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR.typeName());
+        add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException {
-        add(fieldName, String[].class);
+        add(fieldName, PortableClassDescriptor.Mode.STRING_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException {
-        add(fieldName, UUID[].class);
+        add(fieldName, PortableClassDescriptor.Mode.UUID_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException {
-        add(fieldName, Date[].class);
+        add(fieldName, PortableClassDescriptor.Mode.DATE_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException {
-        add(fieldName, Timestamp[].class);
+        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException {
-        add(fieldName, Object[].class);
+        add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
         throws BinaryObjectException {
-        add(fieldName, Collection.class);
+        add(fieldName, PortableClassDescriptor.Mode.COL);
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws BinaryObjectException {
-        add(fieldName, Map.class);
+        add(fieldName, PortableClassDescriptor.Mode.MAP);
     }
 
     /** {@inheritDoc} */
@@ -230,32 +230,23 @@ class BinaryMetaDataCollector implements BinaryWriter {
 
     /**
      * @param name Field name.
-     * @param fieldType Field type.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @param mode Field mode.
+     * @throws BinaryObjectException In case of error.
      */
-    private void add(String name, Class<?> fieldType) throws BinaryObjectException {
-        assert fieldType != null;
-
-        add(name, fieldType.getSimpleName());
-    }
-
-    /**
-     * @param name Field name.
-     * @param fieldTypeName Field type name.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
-     */
-    private void add(String name, String fieldTypeName) throws BinaryObjectException {
+    private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException {
         assert name != null;
 
-        String oldFieldTypeName = meta.put(name, fieldTypeName);
+        int fieldTypeId = mode.typeId();
+
+        Integer oldFieldTypeId = meta.put(name, fieldTypeId);
 
-        if (oldFieldTypeName != null && !oldFieldTypeName.equals(fieldTypeName)) {
+        if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) {
             throw new BinaryObjectException(
                 "Field is written twice with different types [" +
                 "typeName=" + typeName +
                 ", fieldName=" + name +
-                ", fieldTypeName1=" + oldFieldTypeName +
-                ", fieldTypeName2=" + fieldTypeName +
+                ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) +
+                ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) +
                 ']'
             );
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/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 68bc444..6231741 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
@@ -45,7 +45,7 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
 
     /** Recorded object fields. */
     @GridToStringInclude
-    private Map<String, String> fields;
+    private Map<String, Integer> fields;
 
     /** Affinity key field name. */
     private String affKeyFieldName;
@@ -65,7 +65,7 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
      * @param fields Fields map.
      * @param affKeyFieldName Affinity key field name.
      */
-    public BinaryMetaDataImpl(int typeId, String typeName, @Nullable Map<String, String> fields,
+    public BinaryMetaDataImpl(int typeId, String typeName, @Nullable Map<String, Integer> fields,
         @Nullable String affKeyFieldName) {
         assert typeName != null;
 
@@ -95,13 +95,15 @@ public class BinaryMetaDataImpl implements BinaryType, Externalizable {
     /**
      * @return Fields.
      */
-    public Map<String, String> fields0() {
-        return fields != null ? fields : Collections.<String, String>emptyMap();
+    public Map<String, Integer> fields0() {
+        return fields != null ? fields : Collections.<String, Integer>emptyMap();
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public String fieldTypeName(String fieldName) {
-        return fields != null ? fields.get(fieldName) : null;
+        Integer typeId = fields != null ? fields.get(fieldName) : null;
+
+        return typeId != null ? PortableUtils.fieldTypeName(typeId) : null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index a031213..422506a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -92,7 +92,7 @@ public class PortableClassDescriptor {
     private final Method readResolveMtd;
 
     /** */
-    private final Map<String, String> fieldsMeta;
+    private final Map<String, Integer> fieldsMeta;
 
     /** */
     private final boolean keepDeserialized;
@@ -182,7 +182,7 @@ public class PortableClassDescriptor {
             case UUID_ARR:
             case DATE_ARR:
             case TIMESTAMP_ARR:
-            case OBJ_ARR:
+            case OBJECT_ARR:
             case COL:
             case MAP:
             case MAP_ENTRY:
@@ -210,7 +210,7 @@ public class PortableClassDescriptor {
 
                 ctor = constructor(cls);
                 fields = new ArrayList<>();
-                fieldsMeta = metaDataEnabled ? new HashMap<String, String>() : null;
+                fieldsMeta = metaDataEnabled ? new HashMap<String, Integer>() : null;
 
                 Collection<String> names = new HashSet<>();
                 Collection<Integer> ids = new HashSet<>();
@@ -237,7 +237,7 @@ public class PortableClassDescriptor {
                             fields.add(fieldInfo);
 
                             if (metaDataEnabled)
-                                fieldsMeta.put(name, fieldInfo.fieldMode().typeName());
+                                fieldsMeta.put(name, fieldInfo.fieldMode().typeId());
                         }
                     }
                 }
@@ -283,7 +283,7 @@ public class PortableClassDescriptor {
     /**
      * @return Fields meta data.
      */
-    Map<String, String> fieldsMeta() {
+    Map<String, Integer> fieldsMeta() {
         return fieldsMeta;
     }
 
@@ -490,7 +490,7 @@ public class PortableClassDescriptor {
 
                 break;
 
-            case OBJ_ARR:
+            case OBJECT_ARR:
                 writer.doWriteObjectArray((Object[])obj);
 
                 break;
@@ -812,7 +812,7 @@ public class PortableClassDescriptor {
         else if (cls == Timestamp[].class)
             return Mode.TIMESTAMP_ARR;
         else if (cls.isArray())
-            return cls.getComponentType().isEnum() ? Mode.ENUM_ARR : Mode.OBJ_ARR;
+            return cls.getComponentType().isEnum() ? Mode.ENUM_ARR : Mode.OBJECT_ARR;
         else if (cls == BinaryObjectImpl.class)
             return Mode.PORTABLE_OBJ;
         else if (Binarylizable.class.isAssignableFrom(cls))
@@ -1019,7 +1019,7 @@ public class PortableClassDescriptor {
 
                     break;
 
-                case OBJ_ARR:
+                case OBJECT_ARR:
                     writer.writeObjectArrayField((Object[])val);
 
                     break;
@@ -1210,7 +1210,7 @@ public class PortableClassDescriptor {
 
                     break;
 
-                case OBJ_ARR:
+                case OBJECT_ARR:
                     val = reader.readObjectArray(id);
 
                     break;
@@ -1352,7 +1352,7 @@ public class PortableClassDescriptor {
         TIMESTAMP_ARR(GridPortableMarshaller.TIMESTAMP_ARR),
 
         /** */
-        OBJ_ARR(GridPortableMarshaller.OBJ_ARR),
+        OBJECT_ARR(GridPortableMarshaller.OBJ_ARR),
 
         /** */
         COL(GridPortableMarshaller.COL),
@@ -1398,10 +1398,10 @@ public class PortableClassDescriptor {
         }
 
         /**
-         * @return Type name.
+         * @return Type ID.
          */
-        String typeName() {
-            return PortableUtils.fieldTypeName(typeId);
+        int typeId() {
+            return typeId;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/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 84cc748..9e92f3b 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
@@ -751,7 +751,7 @@ public class PortableContext implements Externalizable {
 
         typeMappers.put(typeName, idMapper);
 
-        Map<String, String> fieldsMeta = null;
+        Map<String, Integer> fieldsMeta = null;
 
         if (cls != null) {
             PortableClassDescriptor desc = new PortableClassDescriptor(
@@ -815,7 +815,7 @@ public class PortableContext implements Externalizable {
      * @param fields Fields map.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws BinaryObjectException {
+    public void updateMetaData(int typeId, String typeName, Map<String, Integer> fields) throws BinaryObjectException {
         updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fields, null));
     }
 
@@ -889,6 +889,7 @@ public class PortableContext implements Externalizable {
      * @param clsName Class name.
      * @return Type name.
      */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
     public static String typeName(String clsName) {
         assert clsName != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
index 85f2b68..ca8f09b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
@@ -284,7 +284,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
             if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
                 BinaryType metadata = ctx.metaData(typeId);
 
-                Map<String, String> newFldsMetadata = null;
+                Map<String, Integer> newFldsMetadata = null;
 
                 for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
                     Object val = entry.getValue();
@@ -305,15 +305,14 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
                     String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
 
-                    String newFldTypeName;
+                    int newFldTypeId;
 
                     if (val instanceof PortableValueWithType)
-                        newFldTypeName = ((PortableValueWithType) val).typeName();
-                    else {
-                        byte type = PortableUtils.typeByClass(val.getClass());
+                        newFldTypeId = ((PortableValueWithType) val).typeId();
+                    else
+                        newFldTypeId = PortableUtils.typeByClass(val.getClass());
 
-                        newFldTypeName = PortableUtils.fieldTypeName(type);
-                    }
+                    String newFldTypeName = PortableUtils.fieldTypeName(newFldTypeId);
 
                     if (oldFldTypeName == null) {
                         // It's a new field, we have to add it to metadata.
@@ -321,7 +320,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                         if (newFldsMetadata == null)
                             newFldsMetadata = new HashMap<>();
 
-                        newFldsMetadata.put(name, newFldTypeName);
+                        newFldsMetadata.put(name, PortableUtils.fieldTypeId(newFldTypeName));
                     }
                     else {
                         String objTypeName = PortableUtils.fieldTypeName(GridPortableMarshaller.OBJ);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
index 66d351b..5d66328 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.portable.builder;
 
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
-import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
@@ -48,9 +47,11 @@ class PortableValueWithType implements PortableLazyValue {
             ctx.writeValue(writer, val);
     }
 
-    /** {@inheritDoc} */
-    public String typeName() {
-        return PortableUtils.fieldTypeName(type);
+    /**
+     * @return Type ID.
+     */
+    public int typeId() {
+        return type;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/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 49f4fd8..4ad0762 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
@@ -165,7 +165,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
                         if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) {
                             synchronized (this) {
-                                Map<String, String> fields = new HashMap<>();
+                                Map<String, Integer> fields = new HashMap<>();
 
                                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
                                     newMeta = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
@@ -440,8 +440,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
         Map<String, Integer> fieldTypeIds) throws BinaryObjectException {
-        portableCtx.updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName,
-            PortableUtils.fieldTypeNames(fieldTypeIds), affKeyFieldName));
+        portableCtx.updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fieldTypeIds, affKeyFieldName));
     }
 
     /** {@inheritDoc} */
@@ -716,11 +715,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
     private static boolean checkMeta(int typeId, @Nullable BinaryType oldMeta,
-        BinaryType newMeta, @Nullable Map<String, String> fields) throws BinaryObjectException {
+        BinaryType newMeta, @Nullable Map<String, Integer> fields) throws BinaryObjectException {
         assert newMeta != null;
 
-        Map<String, String> oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fields0() : null;
-        Map<String, String> newFields = ((BinaryMetaDataImpl)newMeta).fields0();
+        Map<String, Integer> oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fields0() : null;
+        Map<String, Integer> newFields = ((BinaryMetaDataImpl)newMeta).fields0();
 
         boolean changed = false;
 
@@ -751,17 +750,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         else
             changed = true;
 
-        for (Map.Entry<String, String> e : newFields.entrySet()) {
-            String typeName = oldFields != null ? oldFields.get(e.getKey()) : null;
+        for (Map.Entry<String, Integer> e : newFields.entrySet()) {
+            Integer oldTypeId = oldFields != null ? oldFields.get(e.getKey()) : null;
 
-            if (typeName != null) {
-                if (!typeName.equals(e.getValue())) {
+            if (oldTypeId != null) {
+                if (!oldTypeId.equals(e.getValue())) {
                     throw new BinaryObjectException(
                         "Portable field has different types on different clients [" +
                             "typeName=" + newMeta.typeName() +
                             ", fieldName=" + e.getKey() +
-                            ", fieldTypeName1=" + typeName +
-                            ", fieldTypeName2=" + e.getValue() +
+                            ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldTypeId) +
+                            ", fieldTypeName2=" + PortableUtils.fieldTypeName(e.getValue()) +
                             ']'
                     );
                 }
@@ -815,7 +814,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             try {
                 BinaryType oldMeta = entry.getValue();
 
-                Map<String, String> fields = new HashMap<>();
+                Map<String, Integer> fields = new HashMap<>();
 
                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
                     BinaryType res = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf108619/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 6dc78e9..af99815 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
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.BinaryMetaDataImpl;
 import org.apache.ignite.internal.portable.BinaryRawReaderEx;
 import org.apache.ignite.internal.portable.BinaryRawWriterEx;
-import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl;
@@ -69,7 +68,6 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformReaderClosur
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T4;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.binary.BinaryType;
 import org.jetbrains.annotations.Nullable;
@@ -392,12 +390,7 @@ public class PlatformContextImpl implements PlatformContext {
         else {
             writer.writeBoolean(true);
 
-            Map<String, String> metaFields = ((BinaryMetaDataImpl)meta).fields0();
-
-            Map<String, Integer> fields = U.newHashMap(metaFields.size());
-
-            for (Map.Entry<String, String> metaField : metaFields.entrySet())
-                fields.put(metaField.getKey(), PortableUtils.fieldTypeId(metaField.getValue()));
+            Map<String, Integer> fields = ((BinaryMetaDataImpl)meta).fields0();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());


[9/9] ignite git commit: IGNITE-1847: Refactoring metadata handlers.

Posted by vo...@apache.org.
IGNITE-1847: Refactoring metadata handlers.


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

Branch: refs/heads/ignite-1847
Commit: b2fd7bfd41817693cdb10fc80c05e3365b177f12
Parents: fc49be0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 10 15:22:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 10 15:22:21 2015 +0300

----------------------------------------------------------------------
 .../portable/BinaryMetaDataCollector.java       | 254 -------------------
 .../internal/portable/BinaryMetaDataImpl.java   | 152 -----------
 .../internal/portable/BinaryMetadata.java       | 152 +++++++++++
 .../portable/BinaryMetadataCollector2.java      | 254 +++++++++++++++++++
 .../portable/BinaryMetadataHandler2.java        |  44 ++++
 .../portable/BinaryNoopMetadataHandler.java     |  53 ++++
 .../internal/portable/BinaryTypeImpl.java       |   6 +-
 .../portable/PortableClassDescriptor.java       |   4 +-
 .../internal/portable/PortableContext.java      |  10 +-
 .../portable/PortableMetaDataHandler.java       |  44 ----
 .../CacheObjectBinaryProcessorImpl.java         |  72 +++---
 .../platform/PlatformContextImpl.java           |   4 +-
 .../PlatformDotNetConfigurationClosure.java     |  19 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |   2 +-
 ...idPortableMarshallerCtxDisabledSelfTest.java |   2 +-
 .../GridPortableMarshallerSelfTest.java         |   2 +-
 .../portable/GridPortableWildcardsSelfTest.java |   2 +-
 .../PortableCompactOffsetsAbstractSelfTest.java |   2 +-
 .../portable/TestCachingMetadataHandler.java    |  45 ++++
 .../internal/portable/TestMetadataHandler.java  |  45 ----
 .../portable/TestNoopMetadataHandler.java       |  36 ---
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   4 +-
 22 files changed, 607 insertions(+), 601 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
deleted file mode 100644
index b9ca926..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryWriter;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Writer for meta data collection.
- */
-class BinaryMetaDataCollector implements BinaryWriter {
-    /** */
-    private final Map<String, Integer> meta = new HashMap<>();
-
-    /** */
-    private final String typeName;
-
-    /**
-     * @param typeName Type name.
-     */
-    BinaryMetaDataCollector(String typeName) {
-        this.typeName = typeName;
-    }
-
-    /**
-     * @return Field meta data.
-     */
-    Map<String, Integer> meta() {
-        return meta;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BYTE);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShort(String fieldName, short val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.SHORT);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeInt(String fieldName, int val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.INT);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLong(String fieldName, long val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.LONG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.FLOAT);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DOUBLE);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChar(String fieldName, char val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.CHAR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DECIMAL);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.STRING);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.UUID);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DATE);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.ENUM);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.OBJECT);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.INT_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.LONG_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.STRING_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.UUID_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DATE_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
-        throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.COL);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.MAP);
-    }
-
-    /** {@inheritDoc} */
-    @Override public BinaryRawWriter rawWriter() {
-        return (BinaryRawWriter)Proxy.newProxyInstance(getClass().getClassLoader(),
-            new Class<?>[] { BinaryRawWriterEx.class },
-            new InvocationHandler() {
-                @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
-                    return null;
-                }
-            });
-    }
-
-    /**
-     * @param name Field name.
-     * @param mode Field mode.
-     * @throws BinaryObjectException In case of error.
-     */
-    private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException {
-        assert name != null;
-
-        int fieldTypeId = mode.typeId();
-
-        Integer oldFieldTypeId = meta.put(name, fieldTypeId);
-
-        if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) {
-            throw new BinaryObjectException(
-                "Field is written twice with different types [" +
-                "typeName=" + typeName +
-                ", fieldName=" + name +
-                ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) +
-                ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) +
-                ']'
-            );
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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
deleted file mode 100644
index a09e14a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Collection;
-import java.util.Collections;
-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.jetbrains.annotations.Nullable;
-
-/**
- * Portable metadata which is passed over a wire.
- */
-public class BinaryMetaDataImpl implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Type ID. */
-    private int typeId;
-
-    /** Type name. */
-    private String typeName;
-
-    /** Recorded object fields. */
-    @GridToStringInclude
-    private Map<String, Integer> fields;
-
-    /** Affinity key field name. */
-    private String affKeyFieldName;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public BinaryMetaDataImpl() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param typeId Type ID.
-     * @param typeName Type name.
-     * @param fields Fields map.
-     * @param affKeyFieldName Affinity key field name.
-     */
-    public BinaryMetaDataImpl(int typeId, String typeName, @Nullable Map<String, Integer> fields,
-        @Nullable String affKeyFieldName) {
-        assert typeName != null;
-
-        this.typeId = typeId;
-        this.typeName = typeName;
-        this.fields = fields;
-        this.affKeyFieldName = affKeyFieldName;
-    }
-
-    /**
-     * @return Type ID.
-     */
-    public int typeId() {
-        return typeId;
-    }
-
-    /**
-     * @return Type name.
-     */
-    public String typeName() {
-        return typeName;
-    }
-
-    /**
-     * @return Fields.
-     */
-    public Collection<String> fields() {
-        return fields != null ? fields.keySet() : Collections.<String>emptyList();
-    }
-
-    /**
-     * @return Fields.
-     */
-    public Map<String, Integer> fieldsMap() {
-        return fields != null ? fields : Collections.<String, Integer>emptyMap();
-    }
-
-    /**
-     * @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;
-    }
-
-    /**
-     * @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);
-        U.writeString(out, typeName);
-        U.writeMap(out, fields);
-        U.writeString(out, affKeyFieldName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        typeId = in.readInt();
-        typeName = U.readString(in);
-        fields = U.readMap(in);
-        affKeyFieldName = U.readString(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(BinaryMetaDataImpl.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
new file mode 100644
index 0000000..fe88d11
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
@@ -0,0 +1,152 @@
+/*
+ * 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 java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collection;
+import java.util.Collections;
+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.jetbrains.annotations.Nullable;
+
+/**
+ * Portable metadata which is passed over a wire.
+ */
+public class BinaryMetadata implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Type ID. */
+    private int typeId;
+
+    /** Type name. */
+    private String typeName;
+
+    /** Recorded object fields. */
+    @GridToStringInclude
+    private Map<String, Integer> fields;
+
+    /** Affinity key field name. */
+    private String affKeyFieldName;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public BinaryMetadata() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param typeId Type ID.
+     * @param typeName Type name.
+     * @param fields Fields map.
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public BinaryMetadata(int typeId, String typeName, @Nullable Map<String, Integer> fields,
+        @Nullable String affKeyFieldName) {
+        assert typeName != null;
+
+        this.typeId = typeId;
+        this.typeName = typeName;
+        this.fields = fields;
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /**
+     * @return Type ID.
+     */
+    public int typeId() {
+        return typeId;
+    }
+
+    /**
+     * @return Type name.
+     */
+    public String typeName() {
+        return typeName;
+    }
+
+    /**
+     * @return Fields.
+     */
+    public Collection<String> fields() {
+        return fields != null ? fields.keySet() : Collections.<String>emptyList();
+    }
+
+    /**
+     * @return Fields.
+     */
+    public Map<String, Integer> fieldsMap() {
+        return fields != null ? fields : Collections.<String, Integer>emptyMap();
+    }
+
+    /**
+     * @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;
+    }
+
+    /**
+     * @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);
+        U.writeString(out, typeName);
+        U.writeMap(out, fields);
+        U.writeString(out, affKeyFieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        typeId = in.readInt();
+        typeName = U.readString(in);
+        fields = U.readMap(in);
+        affKeyFieldName = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryMetadata.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java
new file mode 100644
index 0000000..701c619
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java
@@ -0,0 +1,254 @@
+/*
+ * 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 java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Writer for meta data collection.
+ */
+class BinaryMetadataCollector2 implements BinaryWriter {
+    /** */
+    private final Map<String, Integer> meta = new HashMap<>();
+
+    /** */
+    private final String typeName;
+
+    /**
+     * @param typeName Type name.
+     */
+    BinaryMetadataCollector2(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * @return Field meta data.
+     */
+    Map<String, Integer> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.BYTE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(String fieldName, short val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.SHORT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(String fieldName, int val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.INT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(String fieldName, long val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.LONG);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.FLOAT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.DOUBLE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(String fieldName, char val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.CHAR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.DECIMAL);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.STRING);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.UUID);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.DATE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.ENUM);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.OBJECT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.INT_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.LONG_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.STRING_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.UUID_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.DATE_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
+        throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.COL);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws BinaryObjectException {
+        add(fieldName, PortableClassDescriptor.Mode.MAP);
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryRawWriter rawWriter() {
+        return (BinaryRawWriter)Proxy.newProxyInstance(getClass().getClassLoader(),
+            new Class<?>[] { BinaryRawWriterEx.class },
+            new InvocationHandler() {
+                @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
+                    return null;
+                }
+            });
+    }
+
+    /**
+     * @param name Field name.
+     * @param mode Field mode.
+     * @throws BinaryObjectException In case of error.
+     */
+    private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException {
+        assert name != null;
+
+        int fieldTypeId = mode.typeId();
+
+        Integer oldFieldTypeId = meta.put(name, fieldTypeId);
+
+        if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) {
+            throw new BinaryObjectException(
+                "Field is written twice with different types [" +
+                "typeName=" + typeName +
+                ", fieldName=" + name +
+                ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) +
+                ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) +
+                ']'
+            );
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java
new file mode 100644
index 0000000..6e078a1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java
@@ -0,0 +1,44 @@
+/*
+ * 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.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+/**
+ * Portable meta data handler.
+ */
+public interface BinaryMetadataHandler2 {
+    /**
+     * Adds meta data.
+     *
+     * @param typeId Type ID.
+     * @param meta Meta data.
+     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     */
+    public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException;
+
+    /**
+     * Gets meta data for provided type ID.
+     *
+     * @param typeId Type ID.
+     * @return Meta data.
+     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     */
+    public BinaryType metadata(int typeId) throws BinaryObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java
new file mode 100644
index 0000000..fd0c912
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java
@@ -0,0 +1,53 @@
+/*
+ * 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.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+/**
+ * No-op metadata handler.
+ */
+public class BinaryNoopMetadataHandler implements BinaryMetadataHandler2 {
+    /** Cached singleton instance. */
+    private static final BinaryNoopMetadataHandler INSTANCE = new BinaryNoopMetadataHandler();
+
+    /**
+     * @return Instance.
+     */
+    public static BinaryNoopMetadataHandler instance() {
+        return INSTANCE;
+    }
+
+    /**
+     * Private constructor.
+     */
+    private BinaryNoopMetadataHandler() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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
index 9effdce..40b6252 100644
--- 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
@@ -29,7 +29,7 @@ public class BinaryTypeImpl implements BinaryType {
     private final PortableContext ctx;
 
     /** Type metadata. */
-    private final BinaryMetaDataImpl meta;
+    private final BinaryMetadata meta;
 
     /**
      * Constructor.
@@ -37,7 +37,7 @@ public class BinaryTypeImpl implements BinaryType {
      * @param ctx Portable context.
      * @param meta Type  metadata.
      */
-    public BinaryTypeImpl(PortableContext ctx, BinaryMetaDataImpl meta) {
+    public BinaryTypeImpl(PortableContext ctx, BinaryMetadata meta) {
         this.ctx = ctx;
         this.meta = meta;
     }
@@ -69,7 +69,7 @@ public class BinaryTypeImpl implements BinaryType {
     /**
      * @return Metadata.
      */
-    public BinaryMetaDataImpl metadata() {
+    public BinaryMetadata metadata() {
         return meta;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 422506a..6fdc6b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -544,9 +544,9 @@ public class PortableClassDescriptor {
                         writer.popSchema();
                     }
 
-                    if (obj.getClass() != BinaryMetaDataImpl.class
+                    if (obj.getClass() != BinaryMetadata.class
                         && ctx.isMetaDataChanged(typeId, writer.metaDataHashSum())) {
-                        BinaryMetaDataCollector metaCollector = new BinaryMetaDataCollector(typeName);
+                        BinaryMetadataCollector2 metaCollector = new BinaryMetadataCollector2(typeName);
 
                         if (serializer != null)
                             serializer.writeBinary(obj, metaCollector);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 d54e9b4..8af0630 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
@@ -134,7 +134,7 @@ public class PortableContext implements Externalizable {
     private final Map<String, BinaryTypeIdMapper> typeMappers = new ConcurrentHashMap8<>(0);
 
     /** */
-    private PortableMetaDataHandler metaHnd;
+    private BinaryMetadataHandler2 metaHnd;
 
     /** */
     private MarshallerContext marshCtx;
@@ -165,7 +165,7 @@ public class PortableContext implements Externalizable {
      * @param metaHnd Meta data handler.
      * @param igniteCfg Ignite configuration.
      */
-    public PortableContext(PortableMetaDataHandler metaHnd, IgniteConfiguration igniteCfg) {
+    public PortableContext(BinaryMetadataHandler2 metaHnd, IgniteConfiguration igniteCfg) {
         assert metaHnd != null;
         assert igniteCfg != null;
 
@@ -777,7 +777,7 @@ public class PortableContext implements Externalizable {
             descByCls.put(cls, desc);
         }
 
-        metaHnd.addMeta(id, new BinaryMetaDataImpl(id, typeName, fieldsMeta, affKeyFieldName).wrap(this));
+        metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName).wrap(this));
     }
 
     /**
@@ -832,7 +832,7 @@ public class PortableContext implements Externalizable {
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
     public void updateMetaData(int typeId, String typeName, Map<String, Integer> fields) throws BinaryObjectException {
-        updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fields, null));
+        updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fields, null));
     }
 
     /**
@@ -840,7 +840,7 @@ public class PortableContext implements Externalizable {
      * @param meta Meta data.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    public void updateMetaData(int typeId, BinaryMetaDataImpl meta) throws BinaryObjectException {
+    public void updateMetaData(int typeId, BinaryMetadata meta) throws BinaryObjectException {
         metaHnd.addMeta(typeId, meta.wrap(this));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
deleted file mode 100644
index 8b2eef2..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
-
-/**
- * Portable meta data handler.
- */
-public interface PortableMetaDataHandler {
-    /**
-     * Adds meta data.
-     *
-     * @param typeId Type ID.
-     * @param meta Meta data.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
-     */
-    public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException;
-
-    /**
-     * Gets meta data for provided type ID.
-     *
-     * @param typeId Type ID.
-     * @return Meta data.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
-     */
-    public BinaryType metadata(int typeId) throws BinaryObjectException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 25ffbdf..f8634b7 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
@@ -30,13 +30,13 @@ import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
-import org.apache.ignite.internal.portable.BinaryMetaDataImpl;
+import org.apache.ignite.internal.portable.BinaryMetadata;
 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;
+import org.apache.ignite.internal.portable.BinaryMetadataHandler2;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
@@ -110,7 +110,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private final boolean clientNode;
 
     /** */
-    private volatile IgniteCacheProxy<PortableMetaDataKey, BinaryMetaDataImpl> metaDataCache;
+    private volatile IgniteCacheProxy<PortableMetaDataKey, BinaryMetadata> metaDataCache;
 
     /** */
     private final ConcurrentHashMap8<Integer, BinaryTypeImpl> clientMetaDataCache;
@@ -138,7 +138,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private IgniteBinary portables;
 
     /** Metadata updates collected before metadata cache is initialized. */
-    private final Map<Integer, BinaryMetaDataImpl> metaBuf = new ConcurrentHashMap<>();
+    private final Map<Integer, BinaryMetadata> metaBuf = new ConcurrentHashMap<>();
 
     /** */
     private UUID metaCacheQryId;
@@ -159,22 +159,22 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         if (marsh instanceof PortableMarshaller) {
-            PortableMetaDataHandler metaHnd = new PortableMetaDataHandler() {
+            BinaryMetadataHandler2 metaHnd = new BinaryMetadataHandler2() {
                 @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException {
                     assert newMeta != null;
                     assert newMeta instanceof BinaryTypeImpl;
 
-                    BinaryMetaDataImpl newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
+                    BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
 
                     if (metaDataCache == null) {
-                        BinaryMetaDataImpl oldMeta = metaBuf.get(typeId);
+                        BinaryMetadata oldMeta = metaBuf.get(typeId);
 
                         if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) {
                             synchronized (this) {
                                 Map<String, Integer> fields = new HashMap<>();
 
                                 if (checkMeta(typeId, oldMeta, newMeta0, fields)) {
-                                    newMeta0 = new BinaryMetaDataImpl(typeId, newMeta0.typeName(), fields,
+                                    newMeta0 = new BinaryMetadata(typeId, newMeta0.typeName(), fields,
                                         newMeta0.affinityKeyFieldName());
 
                                     metaBuf.put(typeId, newMeta0);
@@ -238,7 +238,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
                 GridCacheQueryManager qryMgr = metaDataCache.context().queries();
 
-                CacheQuery<Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl>> qry =
+                CacheQuery<Map.Entry<PortableMetaDataKey, BinaryMetadata>> qry =
                     qryMgr.createScanQuery(new MetaDataPredicate(), null, false);
 
                 qry.keepAll(false);
@@ -246,9 +246,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
 
                 try {
-                    CacheQueryFuture<Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl>> fut = qry.execute();
+                    CacheQueryFuture<Map.Entry<PortableMetaDataKey, BinaryMetadata>> fut = qry.execute();
 
-                    Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl> next;
+                    Map.Entry<PortableMetaDataKey, BinaryMetadata> next;
 
                     while ((next = fut.next()) != null) {
                         assert next.getKey() != null : next;
@@ -276,7 +276,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         startLatch.countDown();
 
-        for (Map.Entry<Integer, BinaryMetaDataImpl> e : metaBuf.entrySet())
+        for (Map.Entry<Integer, BinaryMetadata> e : metaBuf.entrySet())
             addMeta(e.getKey(), e.getValue().wrap(portableCtx));
 
         metaBuf.clear();
@@ -294,15 +294,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
      * @param key Metadata key.
      * @param newMeta Metadata.
      */
-    private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryMetaDataImpl newMeta) {
+    private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryMetadata newMeta) {
         int key0 = key.typeId();
 
         clientMetaDataCache.compute(key0,
             new ConcurrentHashMap8.BiFun<Integer, BinaryTypeImpl, BinaryTypeImpl>() {
                 @Override public BinaryTypeImpl apply(Integer key, BinaryTypeImpl oldMeta) {
-                    BinaryMetaDataImpl res;
+                    BinaryMetadata res;
 
-                    BinaryMetaDataImpl oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
+                    BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
 
                     try {
                         res = checkMeta(key, oldMeta0, newMeta, null) ? newMeta : oldMeta0;
@@ -450,7 +450,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
         Map<String, Integer> fieldTypeIds) throws BinaryObjectException {
-        portableCtx.updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fieldTypeIds, affKeyFieldName));
+        portableCtx.updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName));
     }
 
     /** {@inheritDoc} */
@@ -458,12 +458,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         assert newMeta != null;
         assert newMeta instanceof BinaryTypeImpl;
 
-        BinaryMetaDataImpl newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
+        BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
 
         final PortableMetaDataKey key = new PortableMetaDataKey(typeId);
 
         try {
-            BinaryMetaDataImpl oldMeta = metaDataCache.localPeek(key);
+            BinaryMetadata oldMeta = metaDataCache.localPeek(key);
 
             if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) {
                 BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta0));
@@ -483,7 +483,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             if (clientNode)
                 return clientMetaDataCache.get(typeId);
             else {
-                BinaryMetaDataImpl meta = metaDataCache.localPeek(new PortableMetaDataKey(typeId));
+                BinaryMetadata meta = metaDataCache.localPeek(new PortableMetaDataKey(typeId));
 
                 return meta != null ? meta.wrap(portableCtx) : null;
             }
@@ -502,11 +502,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             for (Integer typeId : typeIds)
                 keys.add(new PortableMetaDataKey(typeId));
 
-            Map<PortableMetaDataKey, BinaryMetaDataImpl> meta = metaDataCache.getAll(keys);
+            Map<PortableMetaDataKey, BinaryMetadata> meta = metaDataCache.getAll(keys);
 
             Map<Integer, BinaryType> res = U.newHashMap(meta.size());
 
-            for (Map.Entry<PortableMetaDataKey, BinaryMetaDataImpl> e : meta.entrySet())
+            for (Map.Entry<PortableMetaDataKey, BinaryMetadata> e : meta.entrySet())
                 res.put(e.getKey().typeId(), e.getValue().wrap(portableCtx));
 
             return res;
@@ -527,10 +527,10 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             });
         else {
             return F.viewReadOnly(metaDataCache.entrySetx(metaPred),
-                new C1<Cache.Entry<PortableMetaDataKey, BinaryMetaDataImpl>, BinaryType>() {
+                new C1<Cache.Entry<PortableMetaDataKey, BinaryMetadata>, BinaryType>() {
                     private static final long serialVersionUID = 0L;
 
-                    @Override public BinaryType apply(Cache.Entry<PortableMetaDataKey, BinaryMetaDataImpl> e) {
+                    @Override public BinaryType apply(Cache.Entry<PortableMetaDataKey, BinaryMetadata> e) {
                         return e.getValue().wrap(portableCtx);
                     }
                 });
@@ -734,8 +734,8 @@ 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 BinaryMetaDataImpl oldMeta,
-        BinaryMetaDataImpl newMeta, @Nullable Map<String, Integer> fields) throws BinaryObjectException {
+    private static boolean checkMeta(int typeId, @Nullable BinaryMetadata oldMeta,
+        BinaryMetadata newMeta, @Nullable Map<String, Integer> fields) throws BinaryObjectException {
         assert newMeta != null;
 
         Map<String, Integer> oldFields = oldMeta != null ? oldMeta.fieldsMap() : null;
@@ -799,7 +799,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /**
      */
     private static class MetaDataProcessor implements
-        EntryProcessor<PortableMetaDataKey, BinaryMetaDataImpl, BinaryObjectException>, Externalizable {
+        EntryProcessor<PortableMetaDataKey, BinaryMetadata, BinaryObjectException>, Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -807,7 +807,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         private int typeId;
 
         /** */
-        private BinaryMetaDataImpl newMeta;
+        private BinaryMetadata newMeta;
 
         /**
          * For {@link Externalizable}.
@@ -820,7 +820,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
          * @param typeId Type ID.
          * @param newMeta New metadata.
          */
-        private MetaDataProcessor(int typeId, BinaryMetaDataImpl newMeta) {
+        private MetaDataProcessor(int typeId, BinaryMetadata newMeta) {
             assert newMeta != null;
 
             this.typeId = typeId;
@@ -829,15 +829,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         /** {@inheritDoc} */
         @Override public BinaryObjectException process(
-            MutableEntry<PortableMetaDataKey, BinaryMetaDataImpl> entry,
+            MutableEntry<PortableMetaDataKey, BinaryMetadata> entry,
             Object... args) {
             try {
-                BinaryMetaDataImpl oldMeta = entry.getValue();
+                BinaryMetadata oldMeta = entry.getValue();
 
                 Map<String, Integer> fields = new HashMap<>();
 
                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                    BinaryMetaDataImpl res = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields,
+                    BinaryMetadata res = new BinaryMetadata(typeId, newMeta.typeName(), fields,
                         newMeta.affinityKeyFieldName());
 
                     entry.setValue(res);
@@ -861,7 +861,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             typeId = in.readInt();
-            newMeta = (BinaryMetaDataImpl)in.readObject();
+            newMeta = (BinaryMetadata)in.readObject();
         }
 
         /** {@inheritDoc} */
@@ -873,17 +873,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /**
      *
      */
-    class MetaDataEntryListener implements CacheEntryUpdatedListener<PortableMetaDataKey, BinaryMetaDataImpl> {
+    class MetaDataEntryListener implements CacheEntryUpdatedListener<PortableMetaDataKey, BinaryMetadata> {
         /** {@inheritDoc} */
         @Override public void onUpdated(
-            Iterable<CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryMetaDataImpl>> evts)
+            Iterable<CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryMetadata>> evts)
             throws CacheEntryListenerException {
-            for (CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryMetaDataImpl> evt : evts) {
+            for (CacheEntryEvent<? extends PortableMetaDataKey, ? extends BinaryMetadata> evt : evts) {
                 assert evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED : evt;
 
                 PortableMetaDataKey key = evt.getKey();
 
-                final BinaryMetaDataImpl newMeta = evt.getValue();
+                final BinaryMetadata newMeta = evt.getValue();
 
                 assert newMeta != null : evt;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 fdc0e0d..05d3515 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
@@ -34,7 +34,7 @@ import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.BinaryMetaDataImpl;
+import org.apache.ignite.internal.portable.BinaryMetadata;
 import org.apache.ignite.internal.portable.BinaryRawReaderEx;
 import org.apache.ignite.internal.portable.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
@@ -390,7 +390,7 @@ public class PlatformContextImpl implements PlatformContext {
         else {
             writer.writeBoolean(true);
 
-            Map<String, Integer> fields = ((BinaryMetaDataImpl)meta).fieldsMap();
+            Map<String, Integer> fields = ((BinaryMetadata)meta).fieldsMap();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 60c0693..d0462e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -22,10 +22,10 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PlatformConfiguration;
 import org.apache.ignite.internal.MarshallerContextImpl;
+import org.apache.ignite.internal.portable.BinaryNoopMetadataHandler;
+import org.apache.ignite.internal.portable.BinaryRawWriterEx;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.PortableMetaDataHandler;
-import org.apache.ignite.internal.portable.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure;
 import org.apache.ignite.internal.processors.platform.lifecycle.PlatformLifecycleBean;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
@@ -36,11 +36,9 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
 import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -228,16 +226,7 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
     @SuppressWarnings("deprecation")
     private static GridPortableMarshaller marshaller() {
         try {
-            PortableContext ctx = new PortableContext(new PortableMetaDataHandler() {
-                @Override public void addMeta(int typeId, BinaryType meta)
-                    throws BinaryObjectException {
-                    // No-op.
-                }
-
-                @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
-                    return null;
-                }
-            }, new IgniteConfiguration());
+            PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration());
 
             PortableMarshaller marsh = new PortableMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index 1ee36d8..14fc6f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
@@ -47,7 +47,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     protected static PortableMarshaller createMarshaller() throws Exception {
-        PortableContext ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration());
+        PortableContext ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration());
 
         PortableMarshaller marsh = new PortableMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index d10d3f3..747f8ea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -45,7 +45,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
         PortableMarshaller marsh = new PortableMarshaller();
         marsh.setContext(new MarshallerContextWithNoStorage());
 
-        PortableContext context = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
+        PortableContext context = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index d7007de..a03e73d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -2403,7 +2403,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
         IgniteConfiguration iCfg = new IgniteConfiguration();
 
-        PortableContext ctx = new PortableContext(new TestNoopMetadataHandler(), iCfg);
+        PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), iCfg);
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
index ee90beb..1b036d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
@@ -36,7 +36,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
      * @return Portable context.
      */
     private PortableContext portableContext() {
-        return new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
+        return new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
index 4fa2324..9225b97 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
@@ -47,7 +47,7 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration());
+        ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration());
 
         marsh = new PortableMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
new file mode 100644
index 0000000..cf31c68
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
@@ -0,0 +1,45 @@
+/*
+ * 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.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Test metadata handler.
+ */
+public class TestCachingMetadataHandler implements BinaryMetadataHandler2 {
+    /** Cached metadatas. */
+    private final ConcurrentHashMap<Integer, BinaryType> metas = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
+        BinaryType otherType = metas.put(typeId, meta);
+
+        if (otherType != null)
+            throw new IllegalStateException("Metadata replacement is not allowed in " +
+                TestCachingMetadataHandler.class.getSimpleName() + '.');
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return metas.get(typeId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
deleted file mode 100644
index c25028d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
-
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * Test metadata handler.
- */
-public class TestMetadataHandler implements PortableMetaDataHandler {
-    /** Cached metadatas. */
-    private final ConcurrentHashMap<Integer, BinaryType> metas = new ConcurrentHashMap<>();
-
-    /** {@inheritDoc} */
-    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
-        BinaryType otherType = metas.put(typeId, meta);
-
-        if (otherType != null)
-            throw new IllegalStateException("Metadata replacement is not allowed in " +
-                TestMetadataHandler.class.getSimpleName() + '.');
-    }
-
-    /** {@inheritDoc} */
-    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
-        return metas.get(typeId);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
deleted file mode 100644
index c504365..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
-
-/**
- * No-op metadata handler.
- */
-public class TestNoopMetadataHandler implements PortableMetaDataHandler {
-    /** {@inheritDoc} */
-    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
index c7a80e3..2e868da 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.processors.cache.portable;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.BinaryNoopMetadataHandler;
 import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.TestNoopMetadataHandler;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.Marshaller;
@@ -37,7 +37,7 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 
-        PortableContext pCtx = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
+        PortableContext pCtx = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", pCtx);
 


[8/9] ignite git commit: IGNITE-1847: Refactoring metadata handlers.

Posted by vo...@apache.org.
IGNITE-1847: Refactoring metadata handlers.


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

Branch: refs/heads/ignite-1847
Commit: fc49be04494b80d202db564d766e8aa9526042b8
Parents: a6e2148
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 10 15:10:00 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 10 15:10:00 2015 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    |  2 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  | 24 +++--------
 ...idPortableMarshallerCtxDisabledSelfTest.java | 33 +++++---------
 .../GridPortableMarshallerSelfTest.java         | 13 +-----
 .../portable/GridPortableWildcardsSelfTest.java | 23 +++-------
 .../PortableCompactOffsetsAbstractSelfTest.java | 21 +++------
 .../internal/portable/TestMetadataHandler.java  | 45 ++++++++++++++++++++
 .../portable/TestNoopMetadataHandler.java       | 36 ++++++++++++++++
 ...ridPortableCacheEntryMemorySizeSelfTest.java | 14 +-----
 9 files changed, 113 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/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 c4deaa0..4d1145d 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
@@ -1875,7 +1875,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             if (field0 == null)
             {
-                field0 = obj.fieldDescriptor(propName);
+                field0 = obj.type().field(propName);
 
                 assert field0 != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index 4fa80b4..1ee36d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
@@ -17,15 +17,14 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.math.BigDecimal;
@@ -38,17 +37,6 @@ import java.util.UUID;
  * Contains tests for portable object fields.
  */
 public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTest {
-    /** Dummy metadata handler. */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /** Marshaller. */
     protected PortableMarshaller dfltMarsh;
 
@@ -59,7 +47,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     protected static PortableMarshaller createMarshaller() throws Exception {
-        PortableContext ctx = new PortableContext(META_HND, new IgniteConfiguration());
+        PortableContext ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration());
 
         PortableMarshaller marsh = new PortableMarshaller();
 
@@ -485,7 +473,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
 
         BinaryObjectEx portObj = toPortable(marsh, obj);
 
-        BinaryField field = portObj.fieldDescriptor(fieldName);
+        BinaryField field = portObj.type().field(fieldName);
 
         return new TestContext(obj, portObj, field);
     }
@@ -508,7 +496,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
 
         assert portObj != null;
 
-        BinaryField field = portObj.fieldDescriptor(fieldName);
+        BinaryField field = portObj.type().field(fieldName);
 
         return new TestContext(obj, portObj, field);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index ee91167..d10d3f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -17,38 +17,27 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Arrays;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.MarshallerContextAdapter;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Arrays;
+
 /**
  *
  */
 public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest {
-    /** */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @throws Exception If failed.
      */
@@ -56,7 +45,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
         PortableMarshaller marsh = new PortableMarshaller();
         marsh.setContext(new MarshallerContextWithNoStorage());
 
-        PortableContext context = new PortableContext(META_HND, new IgniteConfiguration());
+        PortableContext context = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index ea96842..d7007de 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -82,17 +82,6 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /** */
     protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
 
-    /** */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @throws Exception If failed.
      */
@@ -2414,7 +2403,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
         IgniteConfiguration iCfg = new IgniteConfiguration();
 
-        PortableContext ctx = new PortableContext(META_HND, iCfg);
+        PortableContext ctx = new PortableContext(new TestNoopMetadataHandler(), iCfg);
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
index 22125af..ee90beb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
@@ -17,37 +17,26 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.util.Arrays;
-import java.util.Map;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryTypeIdMapper;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryTypeIdMapper;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.Arrays;
+import java.util.Map;
+
 /**
  * Wildcards test.
  */
 public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @return Portable context.
      */
     private PortableContext portableContext() {
-        return new PortableContext(META_HND, new IgniteConfiguration());
+        return new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
index db3c821..4fa2324 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.util.Arrays;
 import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -27,6 +25,8 @@ import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.Arrays;
+
 /**
  * Contains tests for compact offsets.
  */
@@ -37,17 +37,6 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
     /** 2 pow 16. */
     private static int POW_16 = 1 << 16;
 
-    /** Dummy metadata handler. */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /** Marshaller. */
     protected PortableMarshaller marsh;
 
@@ -58,7 +47,7 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        ctx = new PortableContext(META_HND, new IgniteConfiguration());
+        ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration());
 
         marsh = new PortableMarshaller();
 
@@ -139,8 +128,8 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
         assert obj.field2 == field2;
 
         // 2. Test fields API.
-        BinaryField field1Desc = portObj.fieldDescriptor("field1");
-        BinaryField field2Desc = portObj.fieldDescriptor("field2");
+        BinaryField field1Desc = portObj.type().field("field1");
+        BinaryField field2Desc = portObj.type().field("field2");
 
         assert field1Desc.exists(portObj);
         assert field2Desc.exists(portObj);

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
new file mode 100644
index 0000000..c25028d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
@@ -0,0 +1,45 @@
+/*
+ * 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.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Test metadata handler.
+ */
+public class TestMetadataHandler implements PortableMetaDataHandler {
+    /** Cached metadatas. */
+    private final ConcurrentHashMap<Integer, BinaryType> metas = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
+        BinaryType otherType = metas.put(typeId, meta);
+
+        if (otherType != null)
+            throw new IllegalStateException("Metadata replacement is not allowed in " +
+                TestMetadataHandler.class.getSimpleName() + '.');
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return metas.get(typeId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
new file mode 100644
index 0000000..c504365
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
@@ -0,0 +1,36 @@
+/*
+ * 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.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+/**
+ * No-op metadata handler.
+ */
+public class TestNoopMetadataHandler implements PortableMetaDataHandler {
+    /** {@inheritDoc} */
+    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
index d277801..c7a80e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
@@ -20,14 +20,12 @@ package org.apache.ignite.internal.processors.cache.portable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.PortableMetaDataHandler;
+import org.apache.ignite.internal.portable.TestNoopMetadataHandler;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
 
 /**
  *
@@ -39,15 +37,7 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 
-        PortableContext pCtx = new PortableContext(new PortableMetaDataHandler() {
-            @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
-                // No-op
-            }
-
-            @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
-                return null;
-            }
-        }, new IgniteConfiguration());
+        PortableContext pCtx = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", pCtx);
 


[4/9] ignite git commit: IGNITE-1847: Refactored field type names (finished).

Posted by vo...@apache.org.
IGNITE-1847: Refactored field type names (finished).


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

Branch: refs/heads/ignite-1847
Commit: fde2b3ec70fb5746271c03969d1a9d2d5b9caee9
Parents: b4c787b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Nov 9 16:41:04 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Nov 9 16:41:04 2015 +0300

----------------------------------------------------------------------
 .../portable/PortableClassDescriptor.java       | 143 +++++++++----------
 .../ignite/internal/portable/PortableUtils.java |  16 ---
 2 files changed, 71 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fde2b3ec/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index dd04828..a031213 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -337,6 +337,7 @@ public class PortableClassDescriptor {
     /**
      * @return portableReadResolve() method
      */
+    @SuppressWarnings("UnusedDeclaration")
     @Nullable Method getReadResolveMethod() {
         return readResolveMtd;
     }
@@ -430,62 +431,62 @@ public class PortableClassDescriptor {
                 break;
 
             case SHORT_ARR:
-                writer.doWriteShortArray((short[])obj);
+                writer.doWriteShortArray((short[]) obj);
 
                 break;
 
             case INT_ARR:
-                writer.doWriteIntArray((int[])obj);
+                writer.doWriteIntArray((int[]) obj);
 
                 break;
 
             case LONG_ARR:
-                writer.doWriteLongArray((long[])obj);
+                writer.doWriteLongArray((long[]) obj);
 
                 break;
 
             case FLOAT_ARR:
-                writer.doWriteFloatArray((float[])obj);
+                writer.doWriteFloatArray((float[]) obj);
 
                 break;
 
             case DOUBLE_ARR:
-                writer.doWriteDoubleArray((double[])obj);
+                writer.doWriteDoubleArray((double[]) obj);
 
                 break;
 
             case CHAR_ARR:
-                writer.doWriteCharArray((char[])obj);
+                writer.doWriteCharArray((char[]) obj);
 
                 break;
 
             case BOOLEAN_ARR:
-                writer.doWriteBooleanArray((boolean[])obj);
+                writer.doWriteBooleanArray((boolean[]) obj);
 
                 break;
 
             case DECIMAL_ARR:
-                writer.doWriteDecimalArray((BigDecimal[])obj);
+                writer.doWriteDecimalArray((BigDecimal[]) obj);
 
                 break;
 
             case STRING_ARR:
-                writer.doWriteStringArray((String[])obj);
+                writer.doWriteStringArray((String[]) obj);
 
                 break;
 
             case UUID_ARR:
-                writer.doWriteUuidArray((UUID[])obj);
+                writer.doWriteUuidArray((UUID[]) obj);
 
                 break;
 
             case DATE_ARR:
-                writer.doWriteDateArray((Date[])obj);
+                writer.doWriteDateArray((Date[]) obj);
 
                 break;
 
             case TIMESTAMP_ARR:
-                writer.doWriteTimestampArray((Timestamp[])obj);
+                writer.doWriteTimestampArray((Timestamp[]) obj);
 
                 break;
 
@@ -711,6 +712,7 @@ public class PortableClassDescriptor {
      * @return Constructor.
      * @throws org.apache.ignite.binary.BinaryObjectException If constructor doesn't exist.
      */
+    @SuppressWarnings("ConstantConditions")
     @Nullable private static Constructor<?> constructor(Class<?> cls) throws BinaryObjectException {
         assert cls != null;
 
@@ -739,11 +741,8 @@ public class PortableClassDescriptor {
             Method writeObj = cls.getDeclaredMethod("writeObject", ObjectOutputStream.class);
             Method readObj = cls.getDeclaredMethod("readObject", ObjectInputStream.class);
 
-            if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
-                writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
-                use = true;
-            else
-                use = false;
+            use = !Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
+                writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class;
         }
         catch (NoSuchMethodException e) {
             use = false;
@@ -961,62 +960,62 @@ public class PortableClassDescriptor {
                     break;
 
                 case SHORT_ARR:
-                    writer.writeShortArrayField((short[])val);
+                    writer.writeShortArrayField((short[]) val);
 
                     break;
 
                 case INT_ARR:
-                    writer.writeIntArrayField((int[])val);
+                    writer.writeIntArrayField((int[]) val);
 
                     break;
 
                 case LONG_ARR:
-                    writer.writeLongArrayField((long[])val);
+                    writer.writeLongArrayField((long[]) val);
 
                     break;
 
                 case FLOAT_ARR:
-                    writer.writeFloatArrayField((float[])val);
+                    writer.writeFloatArrayField((float[]) val);
 
                     break;
 
                 case DOUBLE_ARR:
-                    writer.writeDoubleArrayField((double[])val);
+                    writer.writeDoubleArrayField((double[]) val);
 
                     break;
 
                 case CHAR_ARR:
-                    writer.writeCharArrayField((char[])val);
+                    writer.writeCharArrayField((char[]) val);
 
                     break;
 
                 case BOOLEAN_ARR:
-                    writer.writeBooleanArrayField((boolean[])val);
+                    writer.writeBooleanArrayField((boolean[]) val);
 
                     break;
 
                 case DECIMAL_ARR:
-                    writer.writeDecimalArrayField((BigDecimal[])val);
+                    writer.writeDecimalArrayField((BigDecimal[]) val);
 
                     break;
 
                 case STRING_ARR:
-                    writer.writeStringArrayField((String[])val);
+                    writer.writeStringArrayField((String[]) val);
 
                     break;
 
                 case UUID_ARR:
-                    writer.writeUuidArrayField((UUID[])val);
+                    writer.writeUuidArrayField((UUID[]) val);
 
                     break;
 
                 case DATE_ARR:
-                    writer.writeDateArrayField((Date[])val);
+                    writer.writeDateArrayField((Date[]) val);
 
                     break;
 
                 case TIMESTAMP_ARR:
-                    writer.writeTimestampArrayField((Timestamp[])val);
+                    writer.writeTimestampArrayField((Timestamp[]) val);
 
                     break;
 
@@ -1275,134 +1274,134 @@ public class PortableClassDescriptor {
     /** */
     enum Mode {
         /** */
-        BYTE("byte"),
+        BYTE(GridPortableMarshaller.BYTE),
 
         /** */
-        SHORT("short"),
+        SHORT(GridPortableMarshaller.SHORT),
 
         /** */
-        INT("int"),
+        INT(GridPortableMarshaller.INT),
 
         /** */
-        LONG("long"),
+        LONG(GridPortableMarshaller.LONG),
 
         /** */
-        FLOAT("float"),
+        FLOAT(GridPortableMarshaller.FLOAT),
 
         /** */
-        DOUBLE("double"),
+        DOUBLE(GridPortableMarshaller.DOUBLE),
 
         /** */
-        CHAR("char"),
+        CHAR(GridPortableMarshaller.CHAR),
 
         /** */
-        BOOLEAN("boolean"),
+        BOOLEAN(GridPortableMarshaller.BOOLEAN),
 
         /** */
-        DECIMAL("decimal"),
+        DECIMAL(GridPortableMarshaller.DECIMAL),
 
         /** */
-        STRING("String"),
+        STRING(GridPortableMarshaller.STRING),
 
         /** */
-        UUID("UUID"),
+        UUID(GridPortableMarshaller.UUID),
 
         /** */
-        DATE("Date"),
+        DATE(GridPortableMarshaller.DATE),
 
         /** */
-        TIMESTAMP("Timestamp"),
+        TIMESTAMP(GridPortableMarshaller.TIMESTAMP),
 
         /** */
-        BYTE_ARR("byte[]"),
+        BYTE_ARR(GridPortableMarshaller.BYTE_ARR),
 
         /** */
-        SHORT_ARR("short[]"),
+        SHORT_ARR(GridPortableMarshaller.SHORT_ARR),
 
         /** */
-        INT_ARR("int[]"),
+        INT_ARR(GridPortableMarshaller.INT_ARR),
 
         /** */
-        LONG_ARR("long[]"),
+        LONG_ARR(GridPortableMarshaller.LONG_ARR),
 
         /** */
-        FLOAT_ARR("float[]"),
+        FLOAT_ARR(GridPortableMarshaller.FLOAT_ARR),
 
         /** */
-        DOUBLE_ARR("double[]"),
+        DOUBLE_ARR(GridPortableMarshaller.DOUBLE_ARR),
 
         /** */
-        CHAR_ARR("char[]"),
+        CHAR_ARR(GridPortableMarshaller.CHAR_ARR),
 
         /** */
-        BOOLEAN_ARR("boolean[]"),
+        BOOLEAN_ARR(GridPortableMarshaller.BOOLEAN_ARR),
 
         /** */
-        DECIMAL_ARR("decimal[]"),
+        DECIMAL_ARR(GridPortableMarshaller.DECIMAL_ARR),
 
         /** */
-        STRING_ARR("String[]"),
+        STRING_ARR(GridPortableMarshaller.STRING_ARR),
 
         /** */
-        UUID_ARR("UUID[]"),
+        UUID_ARR(GridPortableMarshaller.UUID_ARR),
 
         /** */
-        DATE_ARR("Date[]"),
+        DATE_ARR(GridPortableMarshaller.DATE_ARR),
 
         /** */
-        TIMESTAMP_ARR("Timestamp[]"),
+        TIMESTAMP_ARR(GridPortableMarshaller.TIMESTAMP_ARR),
 
         /** */
-        OBJ_ARR("Object[]"),
+        OBJ_ARR(GridPortableMarshaller.OBJ_ARR),
 
         /** */
-        COL("Collection"),
+        COL(GridPortableMarshaller.COL),
 
         /** */
-        MAP("Map"),
+        MAP(GridPortableMarshaller.MAP),
 
         /** */
-        MAP_ENTRY("Entry"),
+        MAP_ENTRY(GridPortableMarshaller.MAP_ENTRY),
 
         /** */
-        PORTABLE_OBJ("Object"),
+        PORTABLE_OBJ(GridPortableMarshaller.OBJ),
 
         /** */
-        ENUM("Enum"),
+        ENUM(GridPortableMarshaller.ENUM),
 
         /** */
-        ENUM_ARR("Enum[]"),
+        ENUM_ARR(GridPortableMarshaller.ENUM_ARR),
 
         /** */
-        CLASS("Class"),
+        CLASS(GridPortableMarshaller.CLASS),
 
         /** */
-        PORTABLE("Object"),
+        PORTABLE(GridPortableMarshaller.PORTABLE_OBJ),
 
         /** */
-        EXTERNALIZABLE("Object"),
+        EXTERNALIZABLE(GridPortableMarshaller.OBJ),
 
         /** */
-        OBJECT("Object"),
+        OBJECT(GridPortableMarshaller.OBJ),
 
         /** */
-        EXCLUSION("Exclusion");
+        EXCLUSION(GridPortableMarshaller.OBJ);
 
         /** */
-        private final String typeName;
+        private final int typeId;
 
         /**
-         * @param typeName Type name.
+         * @param typeId Type ID.
          */
-        Mode(String typeName) {
-            this.typeName = typeName;
+        Mode(int typeId) {
+            this.typeId = typeId;
         }
 
         /**
          * @return Type name.
          */
         String typeName() {
-            return typeName;
+            return PortableUtils.fieldTypeName(typeId);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fde2b3ec/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index a22f1aa..31f2bf9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -559,22 +559,6 @@ public class PortableUtils {
     }
 
     /**
-     * Tells whether provided type is portable or a collection.
-     *
-     * @param cls Class to check.
-     * @return Whether type is portable or a collection.
-     */
-    public static boolean isPortableOrCollectionType(Class<?> cls) {
-        assert cls != null;
-
-        return isPortableType(cls) ||
-            cls == Object[].class ||
-            Collection.class.isAssignableFrom(cls) ||
-            Map.class.isAssignableFrom(cls) ||
-            Map.Entry.class.isAssignableFrom(cls);
-    }
-
-    /**
      * Tells whether provided type is portable.
      *
      * @param cls Class to check.