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

[25/59] [abbrv] ignite git commit: ignite-2065: rename "portable" packages to "binary"

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
new file mode 100644
index 0000000..f5e4e06
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -0,0 +1,587 @@
+/*
+ * 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.binary.builder;
+
+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.BinaryType;
+import org.apache.ignite.internal.binary.BinaryMetadata;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableContext;
+import org.apache.ignite.internal.binary.PortableSchema;
+import org.apache.ignite.internal.binary.PortableSchemaRegistry;
+import org.apache.ignite.internal.binary.BinaryMetadata;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableContext;
+import org.apache.ignite.internal.binary.PortableSchema;
+import org.apache.ignite.internal.binary.PortableSchemaRegistry;
+import org.apache.ignite.internal.binary.PortableUtils;
+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.binary.GridPortableMarshaller.DFLT_HDR_LEN;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.FLAGS_POS;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.HASH_CODE_POS;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.PROTO_VER_POS;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.TYPE_ID_POS;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
+
+/**
+ *
+ */
+public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
+    /** */
+    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;
+
+    /** Flags. */
+    private final short flags;
+
+    /** 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 BinaryObjectBuilderImpl(PortableContext ctx, String clsName) {
+        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
+    }
+
+    /**
+     * @param typeName Type name.
+     * @param ctx Context.
+     * @param typeId Type id.
+     */
+    public BinaryObjectBuilderImpl(PortableContext ctx, int typeId, String typeName) {
+        this.typeId = typeId;
+        this.typeName = typeName;
+        this.ctx = ctx;
+
+        start = -1;
+        flags = -1;
+        reader = null;
+        hdrLen = GridPortableMarshaller.DFLT_HDR_LEN;
+
+        readCache = Collections.emptyMap();
+    }
+
+    /**
+     * @param obj Object to wrap.
+     */
+    public BinaryObjectBuilderImpl(BinaryObjectImpl obj) {
+        this(new PortableBuilderReader(obj), obj.start());
+
+        reader.registerObject(this);
+    }
+
+    /**
+     * @param reader ctx
+     * @param start Start.
+     */
+    BinaryObjectBuilderImpl(PortableBuilderReader reader, int start) {
+        this.reader = reader;
+        this.start = start;
+        this.flags = reader.readShortPositioned(start + GridPortableMarshaller.FLAGS_POS);
+
+        byte ver = reader.readBytePositioned(start + GridPortableMarshaller.PROTO_VER_POS);
+
+        PortableUtils.checkProtocolVersion(ver);
+
+        int typeId = reader.readIntPositioned(start + GridPortableMarshaller.TYPE_ID_POS);
+        ctx = reader.portableContext();
+        hashCode = reader.readIntPositioned(start + GridPortableMarshaller.HASH_CODE_POS);
+
+        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+            int mark = reader.position();
+
+            reader.position(start + GridPortableMarshaller.DFLT_HDR_LEN);
+
+            clsNameToWrite = reader.readString();
+
+            Class cls;
+
+            try {
+                // TODO: IGNITE-1272 - Is class loader needed here?
+                cls = U.forName(clsNameToWrite, null);
+            }
+            catch (ClassNotFoundException e) {
+                throw new BinaryInvalidTypeException("Failed to load the class: " + clsNameToWrite, e);
+            }
+
+            this.typeId = ctx.descriptorForClass(cls, false).typeId();
+
+            registeredType = false;
+
+            hdrLen = reader.position() - mark;
+
+            reader.position(mark);
+        }
+        else {
+            this.typeId = typeId;
+            hdrLen = GridPortableMarshaller.DFLT_HDR_LEN;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryObject build() {
+        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx)) {
+            writer.typeId(typeId);
+
+            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
+
+            serializationCtx.registerObjectWriting(this, 0);
+
+            serializeTo(writer, serializationCtx);
+
+            byte[] arr = writer.array();
+
+            return new BinaryObjectImpl(ctx, arr, 0);
+        }
+    }
+
+    /**
+     * @param writer Writer.
+     * @param serializer Serializer.
+     */
+    void serializeTo(BinaryWriterExImpl writer, PortableBuilderSerializer serializer) {
+        try {
+            writer.preWrite(registeredType ? null : clsNameToWrite);
+
+            Set<Integer> remainsFlds = null;
+
+            if (reader != null) {
+                PortableSchema schema = reader.schema();
+
+                Map<Integer, Object> assignedFldsById;
+
+                if (assignedVals != null) {
+                    assignedFldsById = U.newHashMap(assignedVals.size());
+
+                    for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
+                        int fieldId = ctx.fieldId(typeId, entry.getKey());
+
+                        assignedFldsById.put(fieldId, entry.getValue());
+                    }
+
+                    remainsFlds = assignedFldsById.keySet();
+                }
+                else
+                    assignedFldsById = Collections.emptyMap();
+
+                // Get footer details.
+                int fieldIdLen = PortableUtils.fieldIdLength(flags);
+                int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+
+                IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start);
+
+                int footerPos = footer.get1();
+                int footerEnd = footer.get2();
+
+                // Get raw position.
+                int rawPos = PortableUtils.rawOffsetAbsolute(reader, start);
+
+                // Position reader on data.
+                reader.position(start + hdrLen);
+
+                int idx = 0;
+
+                while (reader.position() < rawPos) {
+                    int fieldId = schema.fieldId(idx++);
+                    int fieldLen =
+                        fieldPositionAndLength(footerPos, footerEnd, rawPos, fieldIdLen, fieldOffsetLen).get2();
+
+                    int postPos = reader.position() + fieldLen; // Position where reader will be placed afterwards.
+
+                    footerPos += fieldIdLen + fieldOffsetLen;
+
+                    if (assignedFldsById.containsKey(fieldId)) {
+                        Object assignedVal = assignedFldsById.remove(fieldId);
+
+                        if (assignedVal != REMOVED_FIELD_MARKER) {
+                            writer.writeFieldId(fieldId);
+
+                            serializer.writeValue(writer, assignedVal);
+                        }
+                    }
+                    else {
+                        int type = fieldLen != 0 ? reader.readByte(0) : 0;
+
+                        if (fieldLen != 0 && !PortableUtils.isPlainArrayType(type) && PortableUtils.isPlainType(type)) {
+                            writer.writeFieldId(fieldId);
+
+                            writer.write(reader.array(), reader.position(), fieldLen);
+                        }
+                        else {
+                            writer.writeFieldId(fieldId);
+
+                            Object val;
+
+                            if (fieldLen == 0)
+                                val = null;
+                            else if (readCache == null) {
+                                val = reader.parseValue();
+
+                                assert reader.position() == postPos;
+                            }
+                            else
+                                val = readCache.get(fieldId);
+
+                            serializer.writeValue(writer, val);
+                        }
+                    }
+
+                    reader.position(postPos);
+                }
+            }
+
+            BinaryType meta = ctx.metadata(typeId);
+
+            Map<String, Integer> fieldsMeta = null;
+
+            if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
+                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
+                    Object val = entry.getValue();
+
+                    if (val == REMOVED_FIELD_MARKER)
+                        continue;
+
+                    String name = entry.getKey();
+
+                    int fieldId = ctx.fieldId(typeId, name);
+
+                    if (remainsFlds != null && !remainsFlds.contains(fieldId))
+                        continue;
+
+                    writer.writeFieldId(fieldId);
+
+                    serializer.writeValue(writer, val);
+
+                    String oldFldTypeName = meta == null ? null : meta.fieldTypeName(name);
+
+                    boolean nullObjField = false;
+
+                    int newFldTypeId;
+
+                    if (val instanceof PortableValueWithType) {
+                        newFldTypeId = ((PortableValueWithType)val).typeId();
+
+                        if (newFldTypeId == GridPortableMarshaller.OBJ && ((PortableValueWithType)val).value() == null)
+                            nullObjField = true;
+                    }
+                    else
+                        newFldTypeId = PortableUtils.typeByClass(val.getClass());
+
+                    String newFldTypeName = PortableUtils.fieldTypeName(newFldTypeId);
+
+                    if (oldFldTypeName == null) {
+                        // It's a new field, we have to add it to metadata.
+                        if (fieldsMeta == null)
+                            fieldsMeta = new HashMap<>();
+
+                        fieldsMeta.put(name, PortableUtils.fieldTypeId(newFldTypeName));
+                    }
+                    else if (!nullObjField) {
+                        String objTypeName = PortableUtils.fieldTypeName(GridPortableMarshaller.OBJ);
+
+                        if (!objTypeName.equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
+                            throw new BinaryObjectException(
+                                "Wrong value has been set [" +
+                                    "typeName=" + (typeName == null ? meta.typeName() : typeName) +
+                                    ", fieldName=" + name +
+                                    ", fieldType=" + oldFldTypeName +
+                                    ", assignedValueType=" + newFldTypeName + ']'
+                            );
+                        }
+                    }
+                }
+            }
+
+            if (reader != null) {
+                // Write raw data if any.
+                int rawOff = PortableUtils.rawOffsetAbsolute(reader, start);
+                int footerStart = PortableUtils.footerStartAbsolute(reader, start);
+
+                if (rawOff < footerStart) {
+                    writer.rawWriter();
+
+                    writer.write(reader.array(), rawOff, footerStart - rawOff);
+                }
+
+                // Shift reader to the end of the object.
+                reader.position(start + PortableUtils.length(reader, start));
+            }
+
+            writer.postWrite(true, registeredType, hashCode);
+
+            // Update metadata if needed.
+            int schemaId = writer.schemaId();
+
+            PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
+
+            if (schemaReg.schema(schemaId) == null) {
+                String typeName = this.typeName;
+
+                if (typeName == null) {
+                    assert meta != null;
+
+                    typeName = meta.typeName();
+                }
+
+                PortableSchema curSchema = writer.currentSchema();
+
+                ctx.updateMetadata(typeId, new BinaryMetadata(typeId, typeName, fieldsMeta,
+                    ctx.affinityKeyFieldName(typeId), Collections.singleton(curSchema), false));
+
+                schemaReg.addSchema(curSchema.schemaId(), curSchema);
+            }
+        }
+        finally {
+            writer.popSchema();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryObjectBuilderImpl hashCode(int hashCode) {
+        this.hashCode = hashCode;
+
+        return this;
+    }
+
+    /**
+     * Get field position and length.
+     *
+     * @param footerPos Field position inside the footer (absolute).
+     * @param footerEnd Footer end (absolute).
+     * @param rawPos Raw data position (absolute).
+     * @param fieldIdLen Field ID length.
+     * @param fieldOffsetLen Field offset length.
+     * @return Tuple with field position and length.
+     */
+    private IgniteBiTuple<Integer, Integer> fieldPositionAndLength(int footerPos, int footerEnd, int rawPos,
+        int fieldIdLen, int fieldOffsetLen) {
+        // Get field offset first.
+        int fieldOffset = PortableUtils.fieldOffsetRelative(reader, footerPos + fieldIdLen, fieldOffsetLen);
+        int fieldPos = start + fieldOffset;
+
+        // Get field length.
+        int fieldLen;
+
+        if (footerPos + fieldIdLen + fieldOffsetLen == footerEnd)
+            // This is the last field, compare to raw offset.
+            fieldLen = rawPos - fieldPos;
+        else {
+            // Field is somewhere in the middle, get difference with the next offset.
+            int nextFieldOffset = PortableUtils.fieldOffsetRelative(reader,
+                footerPos + fieldIdLen + fieldOffsetLen + fieldIdLen, fieldOffsetLen);
+
+            fieldLen = nextFieldOffset - fieldOffset;
+        }
+
+        return F.t(fieldPos, fieldLen);
+    }
+
+    /**
+     * Initialize read cache if needed.
+     */
+    private void ensureReadCacheInit() {
+        assert reader != null;
+
+        if (readCache == null) {
+            int fieldIdLen = PortableUtils.fieldIdLength(flags);
+            int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+
+            PortableSchema schema = reader.schema();
+
+            Map<Integer, Object> readCache = new HashMap<>();
+
+            IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start);
+
+            int footerPos = footer.get1();
+            int footerEnd = footer.get2();
+
+            int rawPos = PortableUtils.rawOffsetAbsolute(reader, start);
+
+            int idx = 0;
+
+            while (footerPos + fieldIdLen < footerEnd) {
+                int fieldId = schema.fieldId(idx++);
+
+                IgniteBiTuple<Integer, Integer> posAndLen =
+                    fieldPositionAndLength(footerPos, footerEnd, rawPos, fieldIdLen, fieldOffsetLen);
+
+                Object val = reader.getValueQuickly(posAndLen.get1(), posAndLen.get2());
+
+                readCache.put(fieldId, val);
+
+                // Shift current footer position.
+                footerPos += fieldIdLen + fieldOffsetLen;
+            }
+
+            this.readCache = readCache;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T 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 (T)PortableUtils.unwrapLazy(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryObjectBuilder setField(String name, Object val0) {
+        Object val = val0 == null ? new PortableValueWithType(PortableUtils.typeByClass(Object.class), null) : val0;
+
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        Object oldVal = assignedVals.put(name, val);
+
+        if (oldVal instanceof PortableValueWithType && val0 != null) {
+            ((PortableValueWithType)oldVal).value(val);
+
+            assignedVals.put(name, oldVal);
+        }
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> BinaryObjectBuilder setField(String name, @Nullable T val, Class<? super T> type) {
+        if (assignedVals == null)
+            assignedVals = new LinkedHashMap<>();
+
+        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryObjectBuilder setField(String name, @Nullable BinaryObjectBuilder 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 BinaryObjectBuilderImpl 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 BinaryObjectBuilderImpl wrap(BinaryObject obj) {
+        BinaryObjectImpl heapObj;
+
+        if (obj instanceof BinaryObjectOffheapImpl)
+            heapObj = (BinaryObjectImpl)((BinaryObjectOffheapImpl)obj).heapCopy();
+        else
+            heapObj = (BinaryObjectImpl)obj;
+
+        return new BinaryObjectBuilderImpl(heapObj);
+    }
+
+    /**
+     * @return Object start position in source array.
+     */
+    int start() {
+        return start;
+    }
+
+    /**
+     * @return Object type id.
+     */
+    public int typeId() {
+        return typeId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableAbstractLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableAbstractLazyValue.java
new file mode 100644
index 0000000..0b6cc0a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableAbstractLazyValue.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.binary.builder;
+
+/**
+ *
+ */
+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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderEnum.java
new file mode 100644
index 0000000..3bb8194
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderEnum.java
@@ -0,0 +1,116 @@
+/*
+ * 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.binary.builder;
+
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+
+/**
+ *
+ */
+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 BinaryInvalidTypeException("Failed to load the class: " + clsName, e);
+            }
+
+            this.typeId = reader.portableContext().descriptorForClass(cls, false).typeId();
+        }
+        else {
+            this.typeId = typeId;
+            this.clsName = null;
+        }
+
+        ordinal = reader.readInt();
+    }
+
+    /**
+     * @return Ordinal.
+     */
+    public int getOrdinal() {
+        return ordinal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl 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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderReader.java
new file mode 100644
index 0000000..2b28e3d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderReader.java
@@ -0,0 +1,856 @@
+/*
+ * 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.binary.builder;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableContext;
+import org.apache.ignite.internal.binary.PortablePositionReadable;
+import org.apache.ignite.internal.binary.PortablePrimitives;
+import org.apache.ignite.internal.binary.PortableSchema;
+import org.apache.ignite.internal.binary.streams.PortableHeapInputStream;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableContext;
+import org.apache.ignite.internal.binary.PortablePositionReadable;
+import org.apache.ignite.internal.binary.PortablePrimitives;
+import org.apache.ignite.internal.binary.PortableSchema;
+import org.apache.ignite.internal.binary.PortableUtils;
+import org.apache.ignite.internal.binary.streams.PortableHeapInputStream;
+
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.NULL;
+import static org.apache.ignite.internal.binary.GridPortableMarshaller.STRING;
+
+/**
+ *
+ */
+public class PortableBuilderReader implements PortablePositionReadable {
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final byte[] arr;
+
+    /** */
+    private final BinaryReaderExImpl reader;
+
+    /** */
+    private final Map<Integer, BinaryObjectBuilderImpl> objMap;
+
+    /** */
+    private int pos;
+
+    /*
+     * Constructor.
+     *
+     * @param objImpl Portable object
+     */
+    PortableBuilderReader(BinaryObjectImpl objImpl) {
+        ctx = objImpl.context();
+        arr = objImpl.array();
+        pos = objImpl.start();
+
+        // TODO: IGNITE-1272 - Is class loader needed here?
+        reader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, pos), null);
+
+        objMap = new HashMap<>();
+    }
+
+    /**
+     * Copying constructor.
+     *
+     * @param other Other reader.
+     * @param start Start position.
+     */
+    PortableBuilderReader(PortableBuilderReader other, int start) {
+        this.ctx = other.ctx;
+        this.arr = other.arr;
+        this.pos = start;
+
+        reader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, other.reader.handles());
+
+        this.objMap = other.objMap;
+    }
+
+    /**
+     * @return Portable context.
+     */
+    public PortableContext portableContext() {
+        return ctx;
+    }
+
+    /**
+     * @param obj Mutable portable object.
+     */
+    public void registerObject(BinaryObjectBuilderImpl obj) {
+        objMap.put(obj.start(), obj);
+    }
+
+    /**
+     * Get schema of the object, starting at the given position.
+     *
+     * @return Object's schema.
+     */
+    public PortableSchema schema() {
+        return reader.getOrCreateSchema();
+    }
+
+    /**
+     * @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 PortablePrimitives.readInt(arr, pos + off);
+    }
+
+    /**
+     * @param pos Position in the source array.
+     * @return Read byte value.
+     */
+    public byte readBytePositioned(int pos) {
+        return PortablePrimitives.readByte(arr, pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShortPositioned(int pos) {
+        return PortablePrimitives.readShort(arr, pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readIntPositioned(int pos) {
+        return PortablePrimitives.readInt(arr, pos);
+    }
+
+    /**
+     * @return Read length of array.
+     */
+    public int readLength() {
+        return PortablePrimitives.readInt(arr, pos);
+    }
+
+    /**
+     * Read string length.
+     *
+     * @return String length.
+     */
+    public int readStringLength() {
+        return PortablePrimitives.readInt(arr, pos);
+    }
+
+    /**
+     * Reads string.
+     *
+     * @return String.
+     */
+    public String readString() {
+        byte flag = readByte();
+
+        if (flag == GridPortableMarshaller.NULL)
+            return null;
+
+        if (flag != GridPortableMarshaller.STRING)
+            throw new BinaryObjectException("Failed to deserialize String.");
+
+        int len = readInt();
+
+        String str = new String(arr, pos, len, UTF_8);
+
+        pos += len;
+
+        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;
+
+                break;
+
+            case GridPortableMarshaller.TIMESTAMP:
+                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.TIMESTAMP_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.PORTABLE_OBJ:
+                len = readInt() + 4;
+
+                break;
+
+            default:
+                throw new BinaryObjectException("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 - readIntPositioned(pos + 1);
+
+                BinaryObjectBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new PortableBuilderReader(this, objStart), objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.OBJ: {
+                BinaryObjectBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new PortableBuilderReader(this, pos), pos);
+
+                    objMap.put(pos, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.BYTE:
+                return arr[pos + 1];
+
+            case GridPortableMarshaller.SHORT:
+                return PortablePrimitives.readShort(arr, pos + 1);
+
+            case GridPortableMarshaller.INT:
+                return PortablePrimitives.readInt(arr, pos + 1);
+
+            case GridPortableMarshaller.LONG:
+                return PortablePrimitives.readLong(arr, pos + 1);
+
+            case GridPortableMarshaller.FLOAT:
+                return PortablePrimitives.readFloat(arr, pos + 1);
+
+            case GridPortableMarshaller.DOUBLE:
+                return PortablePrimitives.readDouble(arr, pos + 1);
+
+            case GridPortableMarshaller.CHAR:
+                return PortablePrimitives.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.TIMESTAMP:
+                return new PortablePlainLazyValue(this, pos, len);
+
+            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.TIMESTAMP_ARR:
+            case GridPortableMarshaller.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR:
+            case GridPortableMarshaller.ENUM_ARR:
+            case GridPortableMarshaller.OBJ_ARR:
+            case GridPortableMarshaller.COL:
+            case GridPortableMarshaller.MAP:
+                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 = readIntPositioned(pos + 1);
+
+                int start = readIntPositioned(pos + 4 + size);
+
+                BinaryObjectImpl portableObj = new BinaryObjectImpl(ctx, arr, pos + 4 + start);
+
+                return new PortablePlainPortableObject(portableObj);
+            }
+
+            default:
+                throw new BinaryObjectException("Invalid flag value: " + type);
+        }
+    }
+
+    /**
+     * @return Parsed value.
+     */
+    public Object parseValue() {
+        int valPos = pos;
+
+        byte type = arr[pos++];
+
+        int plainLazyValLen;
+
+        boolean modifiableLazyVal = false;
+
+        switch (type) {
+            case GridPortableMarshaller.NULL:
+                return null;
+
+            case GridPortableMarshaller.HANDLE: {
+                int objStart = pos - 1 - readInt();
+
+                BinaryObjectBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new PortableBuilderReader(this, objStart), objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.OBJ: {
+                pos--;
+
+                BinaryObjectBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new PortableBuilderReader(this, pos), pos);
+
+                    objMap.put(pos, res);
+                }
+
+                pos += readInt(GridPortableMarshaller.TOTAL_LEN_POS);
+
+                return res;
+            }
+
+            case GridPortableMarshaller.BYTE:
+                return arr[pos++];
+
+            case GridPortableMarshaller.SHORT: {
+                Object res = PortablePrimitives.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;
+
+                break;
+
+            case GridPortableMarshaller.TIMESTAMP:
+                plainLazyValLen = 8 + 4;
+
+                break;
+
+            case GridPortableMarshaller.BYTE_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.SHORT_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.INT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.LONG_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.FLOAT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.DOUBLE_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.CHAR_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridPortableMarshaller.BOOLEAN_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                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 BinaryObjectException("Invalid flag value: " + flag);
+
+                    long time = PortablePrimitives.readLong(arr, pos);
+
+                    pos += 8;
+
+                    res[i] = new Date(time);
+                }
+
+                return res;
+            }
+
+            case GridPortableMarshaller.TIMESTAMP_ARR: {
+                int size = readInt();
+
+                Timestamp[] res = new Timestamp[size];
+
+                for (int i = 0; i < res.length; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridPortableMarshaller.NULL)
+                        continue;
+
+                    if (flag != GridPortableMarshaller.TIMESTAMP)
+                        throw new BinaryObjectException("Invalid flag value: " + flag);
+
+                    long time = PortablePrimitives.readLong(arr, pos);
+
+                    pos += 8;
+
+                    int nano = PortablePrimitives.readInt(arr, pos);
+
+                    pos += 4;
+
+                    Timestamp ts = new Timestamp(time);
+
+                    ts.setNanos(ts.getNanos() + nano);
+
+                    res[i] = ts;
+                }
+
+                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; // scale value
+                        pos += 4 + readLength();
+                    }
+                    else
+                        assert flag == GridPortableMarshaller.NULL;
+                }
+
+                return new PortableModifiableLazyValue(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:
+                        return new PortableLazySet(this, size);
+                }
+
+                throw new BinaryObjectException("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.PORTABLE_OBJ: {
+                int size = readInt();
+
+                pos += size;
+
+                int start = readInt();
+
+                BinaryObjectImpl portableObj = new BinaryObjectImpl(ctx, arr,
+                    pos - 4 - size + start);
+
+                return new PortablePlainPortableObject(portableObj);
+            }
+
+            default:
+                throw new BinaryObjectException("Invalid flag value: " + type);
+        }
+
+        PortableAbstractLazyValue res;
+
+        if (modifiableLazyVal)
+            res = new PortableModifiableLazyValue(this, valPos, 1 + plainLazyValLen);
+        else
+            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.
+     */
+    BinaryReaderExImpl 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(BinaryWriterExImpl writer, PortableBuilderSerializer ctx) {
+            ctx.writeValue(writer, wrappedCollection());
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object value() {
+            return PortableUtils.unwrapLazy(wrappedCollection());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializationAware.java
new file mode 100644
index 0000000..03d5720
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializationAware.java
@@ -0,0 +1,31 @@
+/*
+ * 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.binary.builder;
+
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+
+/**
+ *
+ */
+interface PortableBuilderSerializationAware {
+    /**
+     * @param writer Writer.
+     * @param ctx Context.
+     */
+    public void writeTo(BinaryWriterExImpl writer, PortableBuilderSerializer ctx);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializer.java
new file mode 100644
index 0000000..06c51fb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableBuilderSerializer.java
@@ -0,0 +1,217 @@
+/*
+ * 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.binary.builder;
+
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.internal.binary.BinaryMetadata;
+import org.apache.ignite.internal.binary.BinaryObjectExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableContext;
+import org.apache.ignite.internal.binary.PortableUtils;
+
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ *
+ */
+class PortableBuilderSerializer {
+    /** */
+    private final Map<BinaryObjectBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
+
+    /** */
+    private Map<BinaryObject, BinaryObjectBuilderImpl> portableObjToWrapper;
+
+    /**
+     * @param obj Mutable object.
+     * @param posInResArr Object position in the array.
+     */
+    public void registerObjectWriting(BinaryObjectBuilderImpl obj, int posInResArr) {
+        objToPos.put(obj, posInResArr);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param val Value.
+     */
+    public void writeValue(BinaryWriterExImpl writer, Object val) {
+        if (val == null) {
+            writer.writeByte(GridPortableMarshaller.NULL);
+
+            return;
+        }
+
+        if (val instanceof PortableBuilderSerializationAware) {
+            ((PortableBuilderSerializationAware)val).writeTo(writer, this);
+
+            return;
+        }
+
+        if (val instanceof BinaryObjectExImpl) {
+            if (portableObjToWrapper == null)
+                portableObjToWrapper = new IdentityHashMap<>();
+
+            BinaryObjectBuilderImpl wrapper = portableObjToWrapper.get(val);
+
+            if (wrapper == null) {
+                wrapper = BinaryObjectBuilderImpl.wrap((BinaryObject)val);
+
+                portableObjToWrapper.put((BinaryObject)val, wrapper);
+            }
+
+            val = wrapper;
+        }
+
+        if (val instanceof BinaryObjectBuilderImpl) {
+            BinaryObjectBuilderImpl obj = (BinaryObjectBuilderImpl)val;
+
+            Integer posInResArr = objToPos.get(obj);
+
+            if (posInResArr == null) {
+                objToPos.put(obj, writer.out().position());
+
+                obj.serializeTo(writer.newWriter(obj.typeId()), this);
+            }
+            else {
+                int handle = writer.out().position() - posInResArr;
+
+                writer.writeByte(GridPortableMarshaller.HANDLE);
+                writer.writeInt(handle);
+            }
+
+            return;
+        }
+
+        if (val.getClass().isEnum()) {
+            String typeName = PortableContext.typeName(val.getClass().getName());
+            int typeId = writer.context().typeId(typeName);
+
+            BinaryMetadata meta = new BinaryMetadata(typeId, typeName, null, null, null, true);
+            writer.context().updateMetadata(typeId, meta);
+
+            writer.writeByte(GridPortableMarshaller.ENUM);
+            writer.writeInt(typeId);
+            writer.writeInt(((Enum)val).ordinal());
+
+            return;
+        }
+
+        if (val instanceof Collection) {
+            Collection<?> c = (Collection<?>)val;
+
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(c.size());
+
+            byte 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);
+    }
+
+    /**
+     * @param writer Writer.
+     * @param elementType Element type.
+     * @param arr The array.
+     * @param compTypeId Component type ID.
+     */
+    public void writeArray(BinaryWriterExImpl 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(BinaryWriterExImpl 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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableEnumArrayLazyValue.java
new file mode 100644
index 0000000..d6848e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableEnumArrayLazyValue.java
@@ -0,0 +1,116 @@
+/*
+ * 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.binary.builder;
+
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+
+/**
+ *
+ */
+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 BinaryInvalidTypeException("Failed to load the class: " + clsName, e);
+            }
+
+            compTypeId = reader.portableContext().descriptorForClass(cls, true).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 BinaryObjectException("Invalid flag value: " + flag);
+
+            res[i] = new PortableBuilderEnum(reader);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl 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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyArrayList.java
new file mode 100644
index 0000000..846ac82
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyArrayList.java
@@ -0,0 +1,167 @@
+/*
+ * 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.binary.builder;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableUtils;
+
+/**
+ *
+ */
+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.readIntPositioned(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.readIntPositioned(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntPositioned(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);
+
+            int oldPos = reader.position();
+
+            for (Object o : delegate)
+                ctx.writeValue(writer, o);
+
+            // PortableBuilderImpl might have been written. It could override reader's position.
+            reader.position(oldPos);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyLinkedList.java
new file mode 100644
index 0000000..11ed765
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyLinkedList.java
@@ -0,0 +1,218 @@
+/*
+ * 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.binary.builder;
+
+import java.util.AbstractList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableUtils;
+
+/**
+ *
+ */
+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.readIntPositioned(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.readIntPositioned(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(BinaryWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntPositioned(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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyMap.java
new file mode 100644
index 0000000..4bc7622
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/PortableLazyMap.java
@@ -0,0 +1,221 @@
+/*
+ * 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.binary.builder;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridPortableMarshaller;
+import org.apache.ignite.internal.binary.PortableUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class PortableLazyMap extends AbstractMap<Object, Object> implements PortableBuilderSerializationAware {
+    /** */
+    private final PortableBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private Map<Object, Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param off Offset.
+     */
+    private PortableLazyMap(PortableBuilderReader reader, int off) {
+        this.reader = reader;
+        this.off = off;
+    }
+
+    /**
+     * @param reader Reader.
+     * @return PortableLazyMap.
+     */
+    @Nullable public static PortableLazyMap parseMap(PortableBuilderReader reader) {
+        int off = reader.position() - 1;
+
+        int size = reader.readInt();
+
+        reader.skip(1); // map type.
+
+        for (int i = 0; i < size; i++) {
+            reader.skipValue(); // skip key
+            reader.skipValue(); // skip value
+        }
+
+        return new PortableLazyMap(reader, off);
+    }
+
+    /**
+     *
+     */
+    private void ensureDelegateInit() {
+        if (delegate == null) {
+            int size = reader.readIntPositioned(off + 1);
+
+            reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+            delegate = new LinkedHashMap<>();
+
+            for (int i = 0; i < size; i++)
+                delegate.put(PortableUtils.unwrapLazy(reader.parseValue()), reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (delegate == null) {
+            int size = reader.readIntPositioned(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++) {
+                ctx.writeValue(writer, reader.parseValue()); // key
+                ctx.writeValue(writer, reader.parseValue()); // value
+            }
+        }
+        else {
+            writer.writeByte(GridPortableMarshaller.MAP);
+            writer.writeInt(delegate.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+
+            writer.writeByte(colType);
+
+            for (Entry<Object, Object> entry : delegate.entrySet()) {
+                ctx.writeValue(writer, entry.getKey());
+                ctx.writeValue(writer, entry.getValue());
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (delegate == null)
+            return reader.readIntPositioned(off + 1);
+
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKey(Object key) {
+        ensureDelegateInit();
+
+        return delegate.containsKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsValue(Object val) {
+        return values().contains(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Object> keySet() {
+        ensureDelegateInit();
+
+        return delegate.keySet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        if (delegate == null)
+            delegate = new LinkedHashMap<>();
+        else
+            delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(Object key) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.get(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object put(Object key, Object val) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.put(key, val));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(Object key) {
+        ensureDelegateInit();
+
+        return PortableUtils.unwrapLazy(delegate.remove(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Entry<Object, Object>> entrySet() {
+        ensureDelegateInit();
+
+        return new AbstractSet<Entry<Object, Object>>() {
+            @Override public boolean contains(Object o) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override public Iterator<Entry<Object, Object>> iterator() {
+                return new Iterator<Entry<Object, Object>>() {
+                    /** */
+                    private final Iterator<Entry<Object, Object>> itr = delegate.entrySet().iterator();
+
+                    @Override public boolean hasNext() {
+                        return itr.hasNext();
+                    }
+
+                    @Override public Entry<Object, Object> next() {
+                        Entry<Object, Object> res = itr.next();
+
+                        final Object val = res.getValue();
+
+                        if (val instanceof PortableLazyValue) {
+                            return new SimpleEntry<Object, Object>(res.getKey(), val) {
+                                private static final long serialVersionUID = 0L;
+
+                                @Override public Object getValue() {
+                                    return ((PortableLazyValue)val).value();
+                                }
+                            };
+                        }
+
+                        return res;
+                    }
+
+                    @Override public void remove() {
+                        itr.remove();
+                    }
+                };
+            }
+
+            @Override public int size() {
+                return delegate.size();
+            }
+        };
+    }
+}