You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/09/03 23:43:32 UTC

[23/39] ignite git commit: ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller

ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller


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

Branch: refs/heads/ignite-264
Commit: 9057a4c0c7b2242dc9396432d700a87f21a52f1f
Parents: 6769956
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 3 09:06:54 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 3 09:06:54 2015 +0300

----------------------------------------------------------------------
 .../portable/GridPortableMarshaller.java        |  44 +-
 .../portable/PortableAbstractLazyValue.java     |  57 --
 .../internal/portable/PortableBuilderEnum.java  | 114 ---
 .../internal/portable/PortableBuilderImpl.java  | 531 ------------
 .../portable/PortableBuilderReader.java         | 776 ------------------
 .../PortableBuilderSerializationAware.java      |  29 -
 .../portable/PortableBuilderSerializer.java     | 211 -----
 .../portable/PortableClassDescriptor.java       |  57 +-
 .../internal/portable/PortableContext.java      |   6 +-
 .../portable/PortableEnumArrayLazyValue.java    | 112 ---
 .../portable/PortableLazyArrayList.java         | 159 ----
 .../portable/PortableLazyLinkedList.java        | 215 -----
 .../internal/portable/PortableLazyMap.java      | 218 -----
 .../internal/portable/PortableLazyMapEntry.java |  66 --
 .../internal/portable/PortableLazySet.java      |  89 ---
 .../internal/portable/PortableLazyValue.java    |  28 -
 .../portable/PortableObjectArrayLazyValue.java  |  89 ---
 .../portable/PortablePlainLazyValue.java        |  47 --
 .../portable/PortablePlainPortableObject.java   |  50 --
 .../internal/portable/PortableReaderExImpl.java | 154 +++-
 .../ignite/internal/portable/PortableUtils.java |  11 +
 .../portable/PortableValueWithType.java         |  74 --
 .../internal/portable/PortableWriterExImpl.java | 159 +++-
 .../builder/PortableAbstractLazyValue.java      |  57 ++
 .../portable/builder/PortableBuilderEnum.java   | 116 +++
 .../portable/builder/PortableBuilderImpl.java   | 537 +++++++++++++
 .../portable/builder/PortableBuilderReader.java | 800 +++++++++++++++++++
 .../PortableBuilderSerializationAware.java      |  31 +
 .../builder/PortableBuilderSerializer.java      | 214 +++++
 .../builder/PortableEnumArrayLazyValue.java     | 114 +++
 .../portable/builder/PortableLazyArrayList.java | 166 ++++
 .../builder/PortableLazyLinkedList.java         | 217 +++++
 .../portable/builder/PortableLazyMap.java       | 220 +++++
 .../portable/builder/PortableLazyMapEntry.java  |  68 ++
 .../portable/builder/PortableLazySet.java       |  92 +++
 .../portable/builder/PortableLazyValue.java     |  28 +
 .../builder/PortableModifiableLazyValue.java    |  52 ++
 .../builder/PortableObjectArrayLazyValue.java   |  91 +++
 .../builder/PortablePlainLazyValue.java         |  49 ++
 .../builder/PortablePlainPortableObject.java    |  53 ++
 .../portable/builder/PortableValueWithType.java |  75 ++
 .../internal/portable/builder/package-info.java |  22 +
 .../CacheObjectPortableProcessorImpl.java       |   2 +-
 .../resources/META-INF/classnames.properties    |   2 +-
 .../GridPortableBuilderAdditionalSelfTest.java  | 232 +++++-
 .../portable/GridPortableBuilderSelfTest.java   |   1 +
 .../GridPortableMarshallerSelfTest.java         |  72 +-
 .../GridPortableMetaDataDisabledSelfTest.java   |  17 +
 .../portable/GridPortableMetaDataSelfTest.java  |  17 +
 .../mutabletest/GridPortableTestClasses.java    |  38 +-
 ...ClientNodePortableMetadataMultinodeTest.java |  11 +
 51 files changed, 3687 insertions(+), 3003 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index 3b2357e..c7a9e6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -142,67 +142,67 @@ public class GridPortableMarshaller {
     public static final byte OBJ = (byte)103;
 
     /** */
-    static final byte USER_SET = -1;
+    public static final byte USER_SET = -1;
 
     /** */
-    static final byte USER_COL = 0;
+    public static final byte USER_COL = 0;
 
     /** */
-    static final byte ARR_LIST = 1;
+    public static final byte ARR_LIST = 1;
 
     /** */
-    static final byte LINKED_LIST = 2;
+    public static final byte LINKED_LIST = 2;
 
     /** */
-    static final byte HASH_SET = 3;
+    public static final byte HASH_SET = 3;
 
     /** */
-    static final byte LINKED_HASH_SET = 4;
+    public static final byte LINKED_HASH_SET = 4;
 
     /** */
-    static final byte TREE_SET = 5;
+    public static final byte TREE_SET = 5;
 
     /** */
-    static final byte CONC_SKIP_LIST_SET = 6;
+    public static final byte CONC_SKIP_LIST_SET = 6;
 
     /** */
-    static final byte HASH_MAP = 1;
+    public static final byte HASH_MAP = 1;
 
     /** */
-    static final byte LINKED_HASH_MAP = 2;
+    public static final byte LINKED_HASH_MAP = 2;
 
     /** */
-    static final byte TREE_MAP = 3;
+    public static final byte TREE_MAP = 3;
 
     /** */
-    static final byte CONC_HASH_MAP = 4;
+    public static final byte CONC_HASH_MAP = 4;
 
     /** */
-    static final byte PROPERTIES_MAP = 5;
+    public static final byte PROPERTIES_MAP = 5;
 
     /** */
-    static final int OBJECT_TYPE_ID = -1;
+    public static final int OBJECT_TYPE_ID = -1;
 
     /** */
-    static final int UNREGISTERED_TYPE_ID = 0;
+    public static final int UNREGISTERED_TYPE_ID = 0;
 
     /** */
-    static final int TYPE_ID_POS = 2;
+    public static final int TYPE_ID_POS = 2;
 
     /** */
-    static final int HASH_CODE_POS = 6;
+    public static final int HASH_CODE_POS = 6;
 
     /** */
-    static final int TOTAL_LEN_POS = 10;
+    public static final int TOTAL_LEN_POS = 10;
 
     /** */
-    static final byte RAW_DATA_OFF_POS = 14;
+    public static final byte RAW_DATA_OFF_POS = 14;
 
     /** */
-    static final int CLS_NAME_POS = 18;
+    public static final int CLS_NAME_POS = 18;
 
     /** */
-    static final byte DFLT_HDR_LEN = 18;
+    public static final byte DFLT_HDR_LEN = 18;
 
     /** */
     private final PortableContext ctx;
@@ -301,4 +301,4 @@ public class GridPortableMarshaller {
     public PortableContext context() {
         return ctx;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
deleted file mode 100644
index 2b1c4b7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableAbstractLazyValue.java
+++ /dev/null
@@ -1,57 +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;
-
-/**
- *
- */
-abstract class PortableAbstractLazyValue implements PortableLazyValue {
-    /** */
-    protected Object val;
-
-    /** */
-    protected final PortableBuilderReader reader;
-
-    /** */
-    protected final int valOff;
-
-    /**
-     * @param reader Reader.
-     * @param valOff Value.
-     */
-    protected PortableAbstractLazyValue(PortableBuilderReader reader, int valOff) {
-        this.reader = reader;
-        this.valOff = valOff;
-    }
-
-    /**
-     * @return Value.
-     */
-    protected abstract Object init();
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        if (val == null) {
-            val = init();
-
-            assert val != null;
-        }
-
-        return val;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
deleted file mode 100644
index b6ace99..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderEnum.java
+++ /dev/null
@@ -1,114 +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.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-public class PortableBuilderEnum implements PortableBuilderSerializationAware {
-    /** */
-    private final int ordinal;
-
-    /** */
-    private final int typeId;
-
-    /** */
-    private final String clsName;
-
-    /**
-     * @param typeId Type ID.
-     * @param anEnum Enum instance.
-     */
-    public PortableBuilderEnum(int typeId, Enum anEnum) {
-        ordinal = anEnum.ordinal();
-        this.typeId = typeId;
-        clsName = null;
-    }
-
-    /**
-     * @param reader PortableBuilderReader.
-     */
-    public PortableBuilderEnum(PortableBuilderReader reader) {
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            this.typeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            this.typeId = typeId;
-            this.clsName = null;
-        }
-
-        ordinal = reader.readInt();
-    }
-
-    /**
-     * @return Ordinal.
-     */
-    public int getOrdinal() {
-        return ordinal;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        writer.writeByte(GridPortableMarshaller.ENUM);
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-            writer.writeString(clsName);
-        }
-        else
-            writer.writeInt(typeId);
-
-        writer.writeInt(ordinal);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        PortableBuilderEnum that = (PortableBuilderEnum)o;
-
-        return ordinal == that.ordinal && typeId == that.typeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int result = ordinal;
-
-        result = 31 * result + typeId;
-
-        return result;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
deleted file mode 100644
index dac199a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderImpl.java
+++ /dev/null
@@ -1,531 +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.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.GridArgumentCheck;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-
-/**
- *
- */
-public class PortableBuilderImpl implements PortableBuilder {
-    /** */
-    private static final Object REMOVED_FIELD_MARKER = new Object();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final int typeId;
-
-    /** May be null. */
-    private String typeName;
-
-    /** May be null. */
-    private String clsNameToWrite;
-
-    /** */
-    private boolean registeredType = true;
-
-    /** */
-    private Map<String, Object> assignedVals;
-
-    /** */
-    private Map<Integer, Object> readCache;
-
-    /** Position of object in source array, or -1 if object is not created from PortableObject. */
-    private final int start;
-
-    /** Total header length */
-    private final int hdrLen;
-
-    /**
-     * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject.
-     */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private int hashCode;
-
-    /**
-     * @param clsName Class name.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, String clsName) {
-        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
-    }
-
-    /**
-     * @param typeId Type ID.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId) {
-        this(ctx, typeId, null);
-    }
-
-    /**
-     * @param typeName Type name.
-     * @param ctx Context.
-     * @param typeId Type id.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) {
-        this.typeId = typeId;
-        this.typeName = typeName;
-        this.ctx = ctx;
-
-        start = -1;
-        reader = null;
-        hdrLen = DFLT_HDR_LEN;
-
-        readCache = Collections.emptyMap();
-    }
-
-    /**
-     * @param obj Object to wrap.
-     */
-    public PortableBuilderImpl(PortableObjectImpl obj) {
-        this(new PortableBuilderReader(obj), obj.start());
-
-        reader.registerObject(this);
-    }
-
-    /**
-     * @param reader ctx
-     * @param start Start.
-     */
-    PortableBuilderImpl(PortableBuilderReader reader, int start) {
-        this.reader = reader;
-        this.start = start;
-
-        int typeId = reader.readIntAbsolute(start + TYPE_ID_POS);
-        ctx = reader.portableContext();
-        hashCode = reader.readIntAbsolute(start + HASH_CODE_POS);
-
-        if (typeId == UNREGISTERED_TYPE_ID) {
-            int mark = reader.position();
-
-            reader.position(start + CLS_NAME_POS);
-
-            clsNameToWrite = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(clsNameToWrite, null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e);
-            }
-
-            this.typeId = ctx.descriptorForClass(cls).typeId();
-
-            registeredType = false;
-
-            hdrLen = reader.position() - mark;
-
-            reader.position(mark);
-        }
-        else {
-            this.typeId = typeId;
-            hdrLen = DFLT_HDR_LEN;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableObject build() {
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) {
-
-            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
-
-            serializationCtx.registerObjectWriting(this, 0);
-
-            serializeTo(writer, serializationCtx);
-
-            byte[] arr = writer.array();
-
-            return new PortableObjectImpl(ctx, arr, 0);
-        }
-    }
-
-    /**
-     * @param writer Writer.
-     * @param serializer Serializer.
-     */
-    void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) {
-        writer.doWriteByte(GridPortableMarshaller.OBJ);
-        writer.doWriteBoolean(true);
-        writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID);
-        writer.doWriteInt(hashCode);
-
-        // Length and raw offset.
-        writer.reserve(8);
-
-        if (!registeredType)
-            writer.writeString(clsNameToWrite);
-
-
-        Set<Integer> remainsFlds = null;
-
-        if (reader != null) {
-            Map<Integer, Object> assignedFldsById;
-
-            if (assignedVals != null) {
-                assignedFldsById = U.newHashMap(assignedVals.size());
-
-                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                    int fldId = ctx.fieldId(typeId, entry.getKey());
-
-                    assignedFldsById.put(fldId, entry.getValue());
-                }
-
-                remainsFlds = assignedFldsById.keySet();
-            }
-            else
-                assignedFldsById = Collections.emptyMap();
-
-            int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            reader.position(start + hdrLen);
-
-            int cpStart = -1;
-
-            while (reader.position() < rawOff) {
-                int fldId = reader.readInt();
-
-                int len = reader.readInt();
-
-                if (assignedFldsById.containsKey(fldId)) {
-                    if (cpStart >= 0) {
-                        writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart);
-
-                        cpStart = -1;
-                    }
-
-                    Object assignedVal = assignedFldsById.remove(fldId);
-
-                    reader.skip(len);
-
-                    if (assignedVal != REMOVED_FIELD_MARKER) {
-                        writer.writeInt(fldId);
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, assignedVal);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-                else {
-                    if (len != 0 && PortableUtils.isPlainType(reader.readByte(0))) {
-                        if (cpStart < 0)
-                            cpStart = reader.position() - 4 - 4;
-
-                        reader.skip(len);
-                    }
-                    else {
-                        if (cpStart >= 0) {
-                            writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart);
-
-                            cpStart = -1;
-                        }
-                        else
-                            writer.writeInt(fldId);
-
-                        Object val;
-
-                        if (len == 0)
-                            val = null;
-                        else if (readCache == null) {
-                            int savedPos = reader.position();
-
-                            val = reader.parseValue();
-
-                            assert reader.position() == savedPos + len;
-                        }
-                        else {
-                            val = readCache.get(fldId);
-
-                            reader.skip(len);
-                        }
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, val);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-            }
-
-            if (cpStart >= 0)
-                writer.write(reader.array(), cpStart, reader.position() - cpStart);
-        }
-
-        if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
-            boolean metadataEnabled = ctx.isMetaDataEnabled(typeId);
-
-            PortableMetadata metadata = null;
-
-            if (metadataEnabled)
-                metadata = ctx.metaData(typeId);
-
-            Map<String, String> newFldsMetadata = null;
-
-            for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                Object val = entry.getValue();
-
-                if (val == REMOVED_FIELD_MARKER)
-                    continue;
-
-                String name = entry.getKey();
-
-                int fldId = ctx.fieldId(typeId, name);
-
-                if (remainsFlds != null && !remainsFlds.contains(fldId))
-                    continue;
-
-                writer.writeInt(fldId);
-
-                int lenPos = writer.reserveAndMark(4);
-
-                serializer.writeValue(writer, val);
-
-                writer.writeDelta(lenPos);
-
-                if (metadataEnabled) {
-                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
-
-                    String newFldTypeName;
-
-                    if (val instanceof PortableValueWithType)
-                        newFldTypeName = ((PortableValueWithType)val).typeName();
-                    else {
-                        byte type = PortableUtils.typeByClass(val.getClass());
-
-                        newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type);
-                    }
-
-                    if (oldFldTypeName == null) {
-                        // It's a new field, we have to add it to metadata.
-
-                        if (newFldsMetadata == null)
-                            newFldsMetadata = new HashMap<>();
-
-                        newFldsMetadata.put(name, newFldTypeName);
-                    }
-                    else {
-                        if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
-                            throw new PortableException(
-                                "Wrong value has been set [" +
-                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
-                                    ", fieldName=" + name +
-                                    ", fieldType=" + oldFldTypeName +
-                                    ", assignedValueType=" + newFldTypeName +
-                                    ", assignedValue=" + (((PortableValueWithType)val).value()) + ']'
-                            );
-                        }
-                    }
-                }
-            }
-
-            if (newFldsMetadata != null) {
-                String typeName = this.typeName;
-
-                if (typeName == null)
-                    typeName = metadata.typeName();
-
-                ctx.updateMetaData(typeId, typeName, newFldsMetadata);
-            }
-        }
-
-        writer.writeRawOffsetIfNeeded();
-
-        if (reader != null) {
-            int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-            int len = reader.readIntAbsolute(start + TOTAL_LEN_POS);
-
-            if (rawOff < len)
-                writer.write(reader.array(), rawOff, len - rawOff);
-        }
-
-        writer.writeLength();
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilderImpl hashCode(int hashCode) {
-        this.hashCode = hashCode;
-
-        return this;
-    }
-
-    /**
-     *
-     */
-    private void ensureReadCacheInit() {
-        if (readCache == null) {
-            Map<Integer, Object> readCache = new HashMap<>();
-
-            int pos = start + hdrLen;
-            int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            while (pos < end) {
-                int fieldId = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                int len = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                Object val = reader.getValueQuickly(pos, len);
-
-                readCache.put(fieldId, val);
-
-                pos += len;
-            }
-
-            this.readCache = readCache;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <F> F getField(String name) {
-        Object val;
-
-        if (assignedVals != null && assignedVals.containsKey(name)) {
-            val = assignedVals.get(name);
-
-            if (val == REMOVED_FIELD_MARKER)
-                return null;
-        }
-        else {
-            ensureReadCacheInit();
-
-            int fldId = ctx.fieldId(typeId, name);
-
-            val = readCache.get(fldId);
-        }
-
-        return (F)PortableUtils.unwrapLazy(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, Object val) {
-        GridArgumentCheck.notNull(val, "val");
-
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        Object oldVal = assignedVals.put(name, val);
-
-        if (oldVal instanceof PortableValueWithType) {
-            ((PortableValueWithType)oldVal).value(val);
-
-            assignedVals.put(name, oldVal);
-        }
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        //int fldId = ctx.fieldId(typeId, fldName);
-
-        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) {
-        if (builder == null)
-            return setField(name, null, Object.class);
-        else
-            return setField(name, (Object)builder);
-    }
-
-    /**
-     * Removes field from portable object.
-     *
-     * @param name Field name.
-     * @return {@code this} instance for chaining.
-     */
-    @Override public PortableBuilderImpl removeField(String name) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        assignedVals.put(name, REMOVED_FIELD_MARKER);
-
-        return this;
-    }
-
-    /**
-     * Creates builder initialized by specified portable object.
-     *
-     * @param obj Portable object to initialize builder.
-     * @return New builder.
-     */
-    public static PortableBuilderImpl wrap(PortableObject obj) {
-        PortableObjectImpl heapObj;
-
-        if (obj instanceof PortableObjectOffheapImpl)
-            heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy();
-        else
-            heapObj = (PortableObjectImpl)obj;
-
-        return new PortableBuilderImpl(heapObj);
-    }
-
-    /**
-     * @return Object start position in source array.
-     */
-    int start() {
-        return start;
-    }
-
-    /**
-     * @return Object type id.
-     */
-    int typeId() {
-        return typeId;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
deleted file mode 100644
index 30b31f0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderReader.java
+++ /dev/null
@@ -1,776 +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.sql.Timestamp;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.portable.PortableException;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
-
-/**
- *
- */
-class PortableBuilderReader {
-    /** */
-    private static final PortablePrimitives PRIM = PortablePrimitives.get();
-
-    /** */
-    private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final PortableReaderExImpl reader;
-
-    /** */
-    private byte[] arr;
-
-    /** */
-    private int pos;
-
-    /**
-     * @param objImpl Portable object
-     */
-    PortableBuilderReader(PortableObjectImpl objImpl) {
-        ctx = objImpl.context();
-        arr = objImpl.array();
-        pos = objImpl.start();
-
-        // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new PortableReaderExImpl(portableContext(), arr, pos, null);
-    }
-
-    /**
-     * @return Portable context.
-     */
-    public PortableContext portableContext() {
-        return ctx;
-    }
-
-    /**
-     * @param obj Mutable portable object.
-     */
-    public void registerObject(PortableBuilderImpl obj) {
-        objMap.put(obj.start(), obj);
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public int readInt() {
-        int res = readInt(0);
-
-        pos += 4;
-
-        return res;
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public byte readByte() {
-        return arr[pos++];
-    }
-
-    /**
-     * @return Read boolean value.
-     */
-    public boolean readBoolean() {
-        return readByte() == 1;
-    }
-
-    /**
-     * @return Read int value.
-     */
-    public byte readByte(int off) {
-        return arr[pos + off];
-    }
-
-    /**
-     * @param off Offset related to {@link #pos}
-     * @return Read int value.
-     */
-    public int readInt(int off) {
-        return PRIM.readInt(arr, pos + off);
-    }
-
-    /**
-     * @param pos Position in the source array.
-     * @return Read int value.
-     */
-    public int readIntAbsolute(int pos) {
-        return PRIM.readInt(arr, pos);
-    }
-
-    /**
-     * @return Read length of array.
-     */
-    public int readLength() {
-        return PRIM.readInt(arr, pos);
-    }
-
-    /**
-     * Read string length.
-     *
-     * @return String length.
-     */
-    public int readStringLength() {
-        boolean utf = PRIM.readBoolean(arr, pos);
-
-        int arrLen = PRIM.readInt(arr, pos + 1);
-
-        return 1 + (utf ? arrLen : arrLen << 1);
-    }
-
-    /**
-     * Reads string.
-     *
-     * @return String.
-     */
-    public String readString() {
-        byte flag = readByte();
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != STRING)
-            throw new PortableException("Failed to deserialize String.");
-
-        boolean convert = readBoolean();
-        int len = readInt();
-
-        String str;
-
-        if (convert) {
-            str = new String(arr, pos, len, UTF_8);
-
-            pos += len;
-        }
-        else {
-            str = String.valueOf(PRIM.readCharArray(arr, pos, len));
-
-            pos += len << 1;
-        }
-
-        return str;
-    }
-
-    /**
-     *
-     */
-    public void skipValue() {
-        byte type = arr[pos++];
-
-        int len;
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return;
-
-            case GridPortableMarshaller.OBJ:
-                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS - 1) - 1;
-
-                return;
-
-            case GridPortableMarshaller.BOOLEAN:
-            case GridPortableMarshaller.BYTE:
-                len = 1;
-                break;
-
-            case GridPortableMarshaller.CHAR:
-            case GridPortableMarshaller.SHORT:
-                len = 2;
-
-                break;
-
-            case GridPortableMarshaller.HANDLE:
-            case GridPortableMarshaller.FLOAT:
-            case GridPortableMarshaller.INT:
-                len = 4;
-
-                break;
-
-            case GridPortableMarshaller.ENUM:
-                //skipping type id and ordinal value
-                len = 8;
-
-                break;
-
-            case GridPortableMarshaller.LONG:
-            case GridPortableMarshaller.DOUBLE:
-                len = 8;
-
-                break;
-
-            case GridPortableMarshaller.BYTE_ARR:
-            case GridPortableMarshaller.BOOLEAN_ARR:
-                len = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.STRING:
-                len = 4 + readStringLength();
-
-                break;
-
-            case GridPortableMarshaller.DECIMAL:
-                len = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
-
-                break;
-
-            case GridPortableMarshaller.UUID:
-                len = 8 + 8;
-
-                break;
-
-            case GridPortableMarshaller.DATE:
-                len = 8 + 4;
-
-                break;
-
-            case GridPortableMarshaller.CHAR_ARR:
-            case GridPortableMarshaller.SHORT_ARR:
-                len = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.INT_ARR:
-            case GridPortableMarshaller.FLOAT_ARR:
-                len = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.LONG_ARR:
-            case GridPortableMarshaller.DOUBLE_ARR:
-                len = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.DECIMAL_ARR:
-            case GridPortableMarshaller.DATE_ARR:
-            case GridPortableMarshaller.OBJ_ARR:
-            case GridPortableMarshaller.ENUM_ARR:
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR: {
-                int size = readInt();
-
-                for (int i = 0; i < size; i++)
-                    skipValue();
-
-                return;
-            }
-
-            case GridPortableMarshaller.COL: {
-                int size = readInt();
-
-                pos++; // skip collection type
-
-                for (int i = 0; i < size; i++)
-                    skipValue();
-
-                return;
-            }
-
-            case GridPortableMarshaller.MAP: {
-                int size = readInt();
-
-                pos++; // skip collection type
-
-                for (int i = 0; i < size; i++) {
-                    skipValue(); // skip key.
-                    skipValue(); // skip value.
-                }
-
-                return;
-            }
-
-            case GridPortableMarshaller.MAP_ENTRY:
-                skipValue();
-                skipValue();
-
-                return;
-
-            case GridPortableMarshaller.PORTABLE_OBJ:
-                len = readInt() + 4;
-
-                break;
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-
-        pos += len;
-    }
-
-    /**
-     * @param pos Position.
-     * @param len Length.
-     * @return Object.
-     */
-    public Object getValueQuickly(int pos, int len) {
-        byte type = arr[pos];
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return null;
-
-            case GridPortableMarshaller.HANDLE: {
-                int objStart = pos - readIntAbsolute(pos + 1);
-
-                PortableBuilderImpl res = objMap.get(objStart);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
-
-                    objMap.put(objStart, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.OBJ: {
-                PortableBuilderImpl res = objMap.get(pos);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
-
-                    objMap.put(pos, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.BYTE:
-                return arr[pos + 1];
-
-            case GridPortableMarshaller.SHORT:
-                return PRIM.readShort(arr, pos + 1);
-
-            case GridPortableMarshaller.INT:
-                return PRIM.readInt(arr, pos + 1);
-
-            case GridPortableMarshaller.LONG:
-                return PRIM.readLong(arr, pos + 1);
-
-            case GridPortableMarshaller.FLOAT:
-                return PRIM.readFloat(arr, pos + 1);
-
-            case GridPortableMarshaller.DOUBLE:
-                return PRIM.readDouble(arr, pos + 1);
-
-            case GridPortableMarshaller.CHAR:
-                return PRIM.readChar(arr, pos + 1);
-
-            case GridPortableMarshaller.BOOLEAN:
-                return arr[pos + 1] != 0;
-
-            case GridPortableMarshaller.DECIMAL:
-            case GridPortableMarshaller.STRING:
-            case GridPortableMarshaller.UUID:
-            case GridPortableMarshaller.DATE:
-            case GridPortableMarshaller.BYTE_ARR:
-            case GridPortableMarshaller.SHORT_ARR:
-            case GridPortableMarshaller.INT_ARR:
-            case GridPortableMarshaller.LONG_ARR:
-            case GridPortableMarshaller.FLOAT_ARR:
-            case GridPortableMarshaller.DOUBLE_ARR:
-            case GridPortableMarshaller.CHAR_ARR:
-            case GridPortableMarshaller.BOOLEAN_ARR:
-            case GridPortableMarshaller.DECIMAL_ARR:
-            case GridPortableMarshaller.DATE_ARR:
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR:
-                return new PortablePlainLazyValue(this, pos, len);
-
-            case GridPortableMarshaller.COL:
-            case GridPortableMarshaller.OBJ_ARR:
-            case GridPortableMarshaller.MAP:
-            case GridPortableMarshaller.ENUM_ARR:
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new LazyCollection(pos);
-
-            case GridPortableMarshaller.ENUM: {
-                if (len == 1) {
-                    assert readByte(pos) == GridPortableMarshaller.NULL;
-
-                    return null;
-                }
-
-                int mark = position();
-                position(pos + 1);
-
-                PortableBuilderEnum builderEnum = new PortableBuilderEnum(this);
-
-                position(mark);
-
-                return builderEnum;
-            }
-
-            case GridPortableMarshaller.PORTABLE_OBJ: {
-                int size = readIntAbsolute(pos + 1);
-
-                int start = readIntAbsolute(pos + 4 + size);
-
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr, pos + 4 + start);
-
-                return new PortablePlainPortableObject(portableObj);
-            }
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-    }
-
-    /**
-     * @return Parsed value.
-     */
-    public Object parseValue() {
-        int valPos = pos;
-
-        byte type = arr[pos++];
-
-        int plainLazyValLen;
-
-        switch (type) {
-            case GridPortableMarshaller.NULL:
-                return null;
-
-            case GridPortableMarshaller.HANDLE: {
-                int objStart = pos - 1 - readInt();
-
-                PortableBuilderImpl res = objMap.get(objStart);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
-
-                    objMap.put(objStart, res);
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.OBJ: {
-                pos--;
-
-                PortableBuilderImpl res = objMap.get(pos);
-
-                if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
-
-                    objMap.put(pos, res);
-                }
-
-                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS);
-
-                return res;
-            }
-
-            case GridPortableMarshaller.BYTE:
-                return arr[pos++];
-
-            case GridPortableMarshaller.SHORT: {
-                Object res = PRIM.readShort(arr, pos);
-                pos += 2;
-                return res;
-            }
-
-            case GridPortableMarshaller.INT:
-                return readInt();
-
-            case GridPortableMarshaller.LONG:
-                plainLazyValLen = 8;
-
-                break;
-
-            case GridPortableMarshaller.FLOAT:
-                plainLazyValLen = 4;
-
-                break;
-
-            case GridPortableMarshaller.DOUBLE:
-                plainLazyValLen = 8;
-
-                break;
-
-            case GridPortableMarshaller.CHAR:
-                plainLazyValLen = 2;
-
-                break;
-
-            case GridPortableMarshaller.BOOLEAN:
-                return arr[pos++] != 0;
-
-            case GridPortableMarshaller.DECIMAL:
-                plainLazyValLen = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
-
-                break;
-
-            case GridPortableMarshaller.STRING:
-                plainLazyValLen = 4 + readStringLength();
-
-                break;
-
-            case GridPortableMarshaller.UUID:
-                plainLazyValLen = 8 + 8;
-
-                break;
-
-            case GridPortableMarshaller.DATE:
-                plainLazyValLen = 8 + 4;
-
-                break;
-
-            case GridPortableMarshaller.BYTE_ARR:
-                plainLazyValLen = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.SHORT_ARR:
-                plainLazyValLen = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.INT_ARR:
-                plainLazyValLen = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.LONG_ARR:
-                plainLazyValLen = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.FLOAT_ARR:
-                plainLazyValLen = 4 + readLength() * 4;
-
-                break;
-
-            case GridPortableMarshaller.DOUBLE_ARR:
-                plainLazyValLen = 4 + readLength() * 8;
-
-                break;
-
-            case GridPortableMarshaller.CHAR_ARR:
-                plainLazyValLen = 4 + readLength() * 2;
-
-                break;
-
-            case GridPortableMarshaller.BOOLEAN_ARR:
-                plainLazyValLen = 4 + readLength();
-
-                break;
-
-            case GridPortableMarshaller.OBJ_ARR:
-                return new PortableObjectArrayLazyValue(this);
-
-            case GridPortableMarshaller.DATE_ARR: {
-                int size = readInt();
-
-                Date[] res = new Date[size];
-
-                for (int i = 0; i < res.length; i++) {
-                    byte flag = arr[pos++];
-
-                    if (flag == GridPortableMarshaller.NULL) continue;
-
-                    if (flag != GridPortableMarshaller.DATE)
-                        throw new PortableException("Invalid flag value: " + flag);
-
-                    long time = PRIM.readLong(arr, pos);
-
-                    pos += 8;
-
-                    if (ctx.isUseTimestamp()) {
-                        Timestamp ts = new Timestamp(time);
-
-                        ts.setNanos(ts.getNanos() + readInt());
-
-                        res[i] = ts;
-                    }
-                    else {
-                        res[i] = new Date(time);
-
-                        pos += 4;
-                    }
-                }
-
-                return res;
-            }
-
-            case GridPortableMarshaller.UUID_ARR:
-            case GridPortableMarshaller.STRING_ARR:
-            case GridPortableMarshaller.DECIMAL_ARR: {
-                int size = readInt();
-
-                for (int i = 0; i < size; i++) {
-                    byte flag = arr[pos++];
-
-                    if (flag == GridPortableMarshaller.UUID)
-                        pos += 8 + 8;
-                    else if (flag == GridPortableMarshaller.STRING)
-                        pos += 4 + readStringLength();
-                    else if (flag == GridPortableMarshaller.DECIMAL)
-                        pos += 4 + readLength();
-                    else
-                        assert flag == GridPortableMarshaller.NULL;
-                }
-
-                return new PortablePlainLazyValue(this, valPos, pos - valPos);
-            }
-
-            case GridPortableMarshaller.COL: {
-                int size = readInt();
-                byte colType = arr[pos++];
-
-                switch (colType) {
-                    case GridPortableMarshaller.USER_COL:
-                    case GridPortableMarshaller.ARR_LIST:
-                        return new PortableLazyArrayList(this, size);
-
-                    case GridPortableMarshaller.LINKED_LIST:
-                        return new PortableLazyLinkedList(this, size);
-
-                    case GridPortableMarshaller.HASH_SET:
-                    case GridPortableMarshaller.LINKED_HASH_SET:
-                    case GridPortableMarshaller.TREE_SET:
-                    case GridPortableMarshaller.CONC_SKIP_LIST_SET:
-                        return new PortableLazySet(this, size);
-                }
-
-                throw new PortableException("Unknown collection type: " + colType);
-            }
-
-            case GridPortableMarshaller.MAP:
-                return PortableLazyMap.parseMap(this);
-
-            case GridPortableMarshaller.ENUM:
-                return new PortableBuilderEnum(this);
-
-            case GridPortableMarshaller.ENUM_ARR:
-                return new PortableEnumArrayLazyValue(this);
-
-            case GridPortableMarshaller.MAP_ENTRY:
-                return new PortableLazyMapEntry(this);
-
-            case GridPortableMarshaller.PORTABLE_OBJ: {
-                int size = readInt();
-
-                pos += size;
-
-                int start = readInt();
-
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr,
-                    pos - 4 - size + start);
-
-                return new PortablePlainPortableObject(portableObj);
-            }
-
-
-            default:
-                throw new PortableException("Invalid flag value: " + type);
-        }
-
-        PortablePlainLazyValue res = new PortablePlainLazyValue(this, valPos, 1 + plainLazyValLen);
-
-        pos += plainLazyValLen;
-
-        return res;
-    }
-
-    /**
-     * @return Array.
-     */
-    public byte[] array() {
-        return arr;
-    }
-
-    /**
-     * @return Position of reader.
-     */
-    public int position() {
-        return pos;
-    }
-
-    /**
-     * @param pos New pos.
-     */
-    public void position(int pos) {
-        this.pos = pos;
-    }
-
-    /**
-     * @param n Number of bytes to skip.
-     */
-    public void skip(int n) {
-        pos += n;
-    }
-
-    /**
-     * @return Reader.
-     */
-    PortableReaderExImpl reader() {
-        return reader;
-    }
-
-    /**
-     *
-     */
-    private class LazyCollection implements PortableLazyValue {
-        /** */
-        private final int valOff;
-
-        /** */
-        private Object col;
-
-        /**
-         * @param valOff Value.
-         */
-        protected LazyCollection(int valOff) {
-            this.valOff = valOff;
-        }
-
-        /**
-         * @return Object.
-         */
-        private Object wrappedCollection() {
-            if (col == null) {
-                position(valOff);
-
-                col = parseValue();
-            }
-
-            return col;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-            ctx.writeValue(writer, wrappedCollection());
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object value() {
-            return PortableUtils.unwrapLazy(wrappedCollection());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
deleted file mode 100644
index d75cd7b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializationAware.java
+++ /dev/null
@@ -1,29 +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;
-
-/**
- *
- */
-interface PortableBuilderSerializationAware {
-    /**
-     * @param writer Writer.
-     * @param ctx Context.
-     */
-    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
deleted file mode 100644
index 6ce9f4c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableBuilderSerializer.java
+++ /dev/null
@@ -1,211 +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.util.Collection;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import org.apache.ignite.internal.util.GridConcurrentSkipListSet;
-import org.apache.ignite.portable.PortableObject;
-
-/**
- *
- */
-class PortableBuilderSerializer {
-    /** */
-    private final Map<PortableBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
-
-    /** */
-    private Map<PortableObject, PortableBuilderImpl> portableObjToWrapper;
-
-    /**
-     * @param obj Mutable object.
-     * @param posInResArr Object position in the array.
-     */
-    public void registerObjectWriting(PortableBuilderImpl obj, int posInResArr) {
-        objToPos.put(obj, posInResArr);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param val Value.
-     */
-    public void writeValue(PortableWriterExImpl writer, Object val) {
-        if (val == null) {
-            writer.writeByte(GridPortableMarshaller.NULL);
-
-            return;
-        }
-
-        if (val instanceof PortableBuilderSerializationAware) {
-            ((PortableBuilderSerializationAware)val).writeTo(writer, this);
-
-            return;
-        }
-
-        if (val instanceof PortableObjectEx) {
-            if (portableObjToWrapper == null)
-                portableObjToWrapper = new IdentityHashMap<>();
-
-            PortableBuilderImpl wrapper = portableObjToWrapper.get(val);
-
-            if (wrapper == null) {
-                wrapper = PortableBuilderImpl.wrap((PortableObject)val);
-
-                portableObjToWrapper.put((PortableObject)val, wrapper);
-            }
-
-            val = wrapper;
-        }
-
-        if (val instanceof PortableBuilderImpl) {
-            PortableBuilderImpl obj = (PortableBuilderImpl)val;
-
-            Integer posInResArr = objToPos.get(obj);
-
-            if (posInResArr == null) {
-                objToPos.put(obj, writer.outputStream().position());
-
-                obj.serializeTo(writer.newWriter(obj.typeId()), this);
-            }
-            else {
-                int handle = writer.outputStream().position() - posInResArr;
-
-                writer.writeByte(GridPortableMarshaller.HANDLE);
-                writer.writeInt(handle);
-            }
-
-            return;
-        }
-
-        if (val.getClass().isEnum()) {
-            writer.writeByte(GridPortableMarshaller.ENUM);
-            writer.writeInt(writer.context().typeId(val.getClass().getName()));
-            writer.writeInt(((Enum)val).ordinal());
-
-            return;
-        }
-
-        if (val instanceof Collection) {
-            Collection<?> c = (Collection<?>)val;
-
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(c.size());
-
-            byte colType;
-
-            if (c instanceof GridConcurrentSkipListSet)
-                colType = GridPortableMarshaller.CONC_SKIP_LIST_SET;
-            else
-                colType = writer.context().collectionType(c.getClass());
-
-
-            writer.writeByte(colType);
-
-            for (Object obj : c)
-                writeValue(writer, obj);
-
-            return;
-        }
-
-        if (val instanceof Map) {
-            Map<?, ?> map = (Map<?, ?>)val;
-
-            writer.writeByte(GridPortableMarshaller.MAP);
-            writer.writeInt(map.size());
-
-            writer.writeByte(writer.context().mapType(map.getClass()));
-
-            for (Map.Entry<?, ?> entry : map.entrySet()) {
-                writeValue(writer, entry.getKey());
-                writeValue(writer, entry.getValue());
-            }
-
-            return;
-        }
-
-        Byte flag = PortableUtils.PLAIN_CLASS_TO_FLAG.get(val.getClass());
-
-        if (flag != null) {
-            PortableUtils.writePlainObject(writer, val);
-
-            return;
-        }
-
-        if (val instanceof Object[]) {
-            int compTypeId = writer.context().typeId(((Object[])val).getClass().getComponentType().getName());
-
-            if (val instanceof PortableBuilderEnum[]) {
-                writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
-
-                return;
-            }
-
-            if (((Object[])val).getClass().getComponentType().isEnum()) {
-                Enum[] enumArr = (Enum[])val;
-
-                writer.writeByte(GridPortableMarshaller.ENUM_ARR);
-                writer.writeInt(compTypeId);
-                writer.writeInt(enumArr.length);
-
-                for (Enum anEnum : enumArr)
-                    writeValue(writer, anEnum);
-
-                return;
-            }
-
-            writeArray(writer, GridPortableMarshaller.OBJ_ARR, (Object[])val, compTypeId);
-
-            return;
-        }
-
-        writer.doWriteObject(val, false);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param elementType Element type.
-     * @param arr The array.
-     * @param compTypeId Component type ID.
-     */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
-        writer.writeByte(elementType);
-        writer.writeInt(compTypeId);
-        writer.writeInt(arr.length);
-
-        for (Object obj : arr)
-            writeValue(writer, obj);
-    }
-
-    /**
-     * @param writer Writer.
-     * @param elementType Element type.
-     * @param arr The array.
-     * @param clsName Component class name.
-     */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
-        writer.writeByte(elementType);
-        writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-        writer.writeString(clsName);
-        writer.writeInt(arr.length);
-
-        for (Object obj : arr)
-            writeValue(writer, obj);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 7e4266c..24ad5ce 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
@@ -48,7 +48,7 @@ import static java.lang.reflect.Modifier.isTransient;
 /**
  * Portable class descriptor.
  */
-class PortableClassDescriptor {
+public class PortableClassDescriptor {
     /** */
     private final PortableContext ctx;
 
@@ -123,7 +123,7 @@ class PortableClassDescriptor {
         boolean keepDeserialized
     ) throws PortableException {
         this(ctx, cls, userType, typeId, typeName, idMapper, serializer, useTs, metaDataEnabled, keepDeserialized,
-             true);
+            true);
     }
 
     /**
@@ -285,7 +285,7 @@ class PortableClassDescriptor {
     /**
      * @return Type ID.
      */
-    int typeId() {
+    public int typeId() {
         return typeId;
     }
 
@@ -399,7 +399,7 @@ class PortableClassDescriptor {
                 break;
 
             case DECIMAL:
-                writer.doWriteDecimal((BigDecimal) obj);
+                writer.doWriteDecimal((BigDecimal)obj);
 
                 break;
 
@@ -656,39 +656,32 @@ class PortableClassDescriptor {
      * @return Whether further write is needed.
      */
     private boolean writeHeader(Object obj, PortableWriterExImpl writer) {
-        int handle = writer.handle(obj);
-
-        if (handle >= 0) {
-            writer.doWriteByte(GridPortableMarshaller.HANDLE);
-            writer.doWriteInt(handle);
-
+        if (writer.tryWriteAsHandle(obj))
             return false;
-        }
-        else {
-            int pos = writer.position();
 
-            writer.doWriteByte(GridPortableMarshaller.OBJ);
-            writer.doWriteBoolean(userType);
-            writer.doWriteInt(registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID);
-            writer.doWriteInt(obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
+        int pos = writer.position();
 
-            // For length and raw offset.
-            int reserved = writer.reserve(8);
+        writer.doWriteByte(GridPortableMarshaller.OBJ);
+        writer.doWriteBoolean(userType);
+        writer.doWriteInt(registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID);
+        writer.doWriteInt(obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
 
-            // Class name in case if typeId registration is failed.
-            if (!registered)
-                writer.doWriteString(cls.getName());
+        // For length and raw offset.
+        int reserved = writer.reserve(8);
 
-            int current = writer.position();
-            int len = current - pos;
+        // Class name in case if typeId registration is failed.
+        if (!registered)
+            writer.doWriteString(cls.getName());
 
-            // Default raw offset (equal to header length).
-            writer.position(reserved + 4);
-            writer.doWriteInt(len);
-            writer.position(current);
+        int current = writer.position();
+        int len = current - pos;
 
-            return true;
-        }
+        // Default raw offset (equal to header length).
+        writer.position(reserved + 4);
+        writer.doWriteInt(len);
+        writer.position(current);
+
+        return true;
     }
 
     /**
@@ -787,7 +780,7 @@ class PortableClassDescriptor {
         else if (cls == PortableObjectImpl.class)
             return Mode.PORTABLE_OBJ;
         else if (PortableMarshalAware.class.isAssignableFrom(cls))
-           return Mode.PORTABLE;
+            return Mode.PORTABLE;
         else if (Externalizable.class.isAssignableFrom(cls))
             return Mode.EXTERNALIZABLE;
         else if (Map.Entry.class.isAssignableFrom(cls))
@@ -1352,4 +1345,4 @@ class PortableClassDescriptor {
             return typeName;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/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 15b6433..db7e41e 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
@@ -794,10 +794,10 @@ public class PortableContext implements Externalizable {
      * @param typeId Type ID.
      * @return Whether meta data is enabled.
      */
-    boolean isMetaDataEnabled(int typeId) {
+    public boolean isMetaDataEnabled(int typeId) {
         Boolean enabled = metaEnabled.get(typeId);
 
-        return enabled != null ? enabled : true;
+        return enabled != null ? enabled : metaDataEnabled;
     }
 
     /**
@@ -827,7 +827,7 @@ public class PortableContext implements Externalizable {
      * @param fields Fields map.
      * @throws PortableException In case of error.
      */
-    void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
+    public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
         updateMetaData(typeId, new PortableMetaDataImpl(typeName, fields, null));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
deleted file mode 100644
index c953bb3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableEnumArrayLazyValue.java
+++ /dev/null
@@ -1,112 +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.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-
-/**
- *
- */
-class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
-    /** */
-    private final int len;
-
-    /** */
-    private final int compTypeId;
-
-    /** */
-    private final String clsName;
-
-    /**
-     * @param reader Reader.
-     */
-    protected PortableEnumArrayLazyValue(PortableBuilderReader reader) {
-        super(reader, reader.position() - 1);
-
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            compTypeId = typeId;
-            clsName = null;
-        }
-
-        int size = reader.readInt();
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-
-        len = reader.position() - valOff;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        reader.position(valOff + 1);
-
-        //skipping component type id
-        reader.readInt();
-
-        int size = reader.readInt();
-
-        PortableBuilderEnum[] res = new PortableBuilderEnum[size];
-
-        for (int i = 0; i < size; i++) {
-            byte flag = reader.readByte();
-
-            if (flag == GridPortableMarshaller.NULL)
-                continue;
-
-            if (flag != GridPortableMarshaller.ENUM)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            res[i] = new PortableBuilderEnum(reader);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (val != null) {
-            if (clsName != null)
-                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, clsName);
-            else
-                ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, compTypeId);
-
-            return;
-        }
-
-        writer.write(reader.array(), valOff, len);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
deleted file mode 100644
index 84a0c22..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyArrayList.java
+++ /dev/null
@@ -1,159 +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.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- *
- */
-class PortableLazyArrayList extends AbstractList<Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private List<Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param size Size,
-     */
-    PortableLazyArrayList(PortableBuilderReader reader, int size) {
-        this.reader = reader;
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new ArrayList<>(size);
-
-            for (int i = 0; i < size; i++)
-                delegate.add(reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean add(Object o) {
-        ensureDelegateInit();
-
-        return delegate.add(o);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void add(int idx, Object element) {
-        ensureDelegateInit();
-
-        delegate.add(idx, element);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object set(int idx, Object element) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.set(idx, element));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new ArrayList<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addAll(int idx, Collection<?> c) {
-        return delegate.addAll(idx, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void removeRange(int fromIdx, int toIdx) {
-        ensureDelegateInit();
-
-        delegate.subList(fromIdx, toIdx).clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : delegate)
-                ctx.writeValue(writer, o);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
deleted file mode 100644
index 7b2e15a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyLinkedList.java
+++ /dev/null
@@ -1,215 +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.util.AbstractList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
-
-/**
- *
- */
-class PortableLazyLinkedList extends AbstractList<Object> implements PortableBuilderSerializationAware {
-    /** */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private final int off;
-
-    /** */
-    private List<Object> delegate;
-
-    /**
-     * @param reader Reader.
-     * @param size Size,
-     */
-    PortableLazyLinkedList(PortableBuilderReader reader, int size) {
-        this.reader = reader;
-        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
-
-        assert size >= 0;
-
-        for (int i = 0; i < size; i++)
-            reader.skipValue();
-    }
-
-    /**
-     *
-     */
-    private void ensureDelegateInit() {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
-
-            delegate = new LinkedList<>();
-
-            for (int i = 0; i < size; i++)
-                delegate.add(reader.parseValue());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.get(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean add(Object o) {
-        ensureDelegateInit();
-
-        return delegate.add(o);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void add(int idx, Object element) {
-        ensureDelegateInit();
-
-        delegate.add(idx, element);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object set(int idx, Object element) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.set(idx, element));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object remove(int idx) {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazy(delegate.remove(idx));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        if (delegate == null)
-            delegate = new LinkedList<>();
-        else
-            delegate.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addAll(int idx, Collection<?> c) {
-        ensureDelegateInit();
-
-        return delegate.addAll(idx, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void removeRange(int fromIdx, int toIdx) {
-        ensureDelegateInit();
-
-        delegate.subList(fromIdx, toIdx).clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size() {
-        if (delegate == null)
-            return reader.readIntAbsolute(off + 1);
-
-        return delegate.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ListIterator<Object> listIterator(final int idx) {
-        ensureDelegateInit();
-
-        return new ListIterator<Object>() {
-            /** */
-            private final ListIterator<Object> delegate = PortableLazyLinkedList.super.listIterator(idx);
-
-            @Override public boolean hasNext() {
-                return delegate.hasNext();
-            }
-
-            @Override public Object next() {
-                return PortableUtils.unwrapLazy(delegate.next());
-            }
-
-            @Override public boolean hasPrevious() {
-                return delegate.hasPrevious();
-            }
-
-            @Override public Object previous() {
-                return PortableUtils.unwrapLazy(delegate.previous());
-            }
-
-            @Override public int nextIndex() {
-                return delegate.nextIndex();
-            }
-
-            @Override public int previousIndex() {
-                return delegate.previousIndex();
-            }
-
-            @Override public void remove() {
-                delegate.remove();
-            }
-
-            @Override public void set(Object o) {
-                delegate.set(o);
-            }
-
-            @Override public void add(Object o) {
-                delegate.add(o);
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Object> iterator() {
-        ensureDelegateInit();
-
-        return PortableUtils.unwrapLazyIterator(super.iterator());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        if (delegate == null) {
-            int size = reader.readIntAbsolute(off + 1);
-
-            int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */;
-            writer.write(reader.array(), off, hdrSize);
-
-            reader.position(off + hdrSize);
-
-            for (int i = 0; i < size; i++) {
-                Object o = reader.parseValue();
-
-                ctx.writeValue(writer, o);
-            }
-        }
-        else {
-            writer.writeByte(GridPortableMarshaller.COL);
-            writer.writeInt(delegate.size());
-
-            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
-            writer.writeByte(colType);
-
-            for (Object o : delegate)
-                ctx.writeValue(writer, o);
-        }
-    }
-}
\ No newline at end of file