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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
new file mode 100644
index 0000000..45355d7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -0,0 +1,800 @@
+/*
+ * 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.builder;
+
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.PortablePrimitives;
+import org.apache.ignite.internal.portable.PortableReaderExImpl;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+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:
+                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.UUID_ARR:
+            case GridPortableMarshaller.STRING_ARR:
+            case GridPortableMarshaller.ENUM_ARR:
+            case GridPortableMarshaller.OBJ_ARR:
+            case GridPortableMarshaller.COL:
+            case GridPortableMarshaller.MAP:
+            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;
+
+        boolean modifiableLazyVal = false;
+
+        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();
+                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 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; // 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:
+                    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);
+        }
+
+        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.
+     */
+    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/builder/PortableBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializationAware.java
new file mode 100644
index 0000000..976059a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/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.portable.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+/**
+ *
+ */
+interface PortableBuilderSerializationAware {
+    /**
+     * @param writer Writer.
+     * @param ctx Context.
+     */
+    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
new file mode 100644
index 0000000..2d9c961
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -0,0 +1,214 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableObjectEx;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.portable.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+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/builder/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
new file mode 100644
index 0000000..d864a6e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
@@ -0,0 +1,114 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+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/builder/PortableLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
new file mode 100644
index 0000000..a08cfdd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyArrayList.java
@@ -0,0 +1,166 @@
+/*
+ * 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.builder;
+
+import 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);
+
+            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/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.java
new file mode 100644
index 0000000..f793d7a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyLinkedList.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.portable.builder;
+
+import 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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
new file mode 100644
index 0000000..12cbfd6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMap.java
@@ -0,0 +1,220 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+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.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.readIntAbsolute(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(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++) {
+                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.readIntAbsolute(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();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
new file mode 100644
index 0000000..bd027f5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyMapEntry.java
@@ -0,0 +1,68 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+import java.util.Map;
+
+/**
+ *
+ */
+class PortableLazyMapEntry implements Map.Entry<Object, Object>, PortableBuilderSerializationAware {
+    /** */
+    private final Object key;
+
+    /** */
+    private Object val;
+
+    /**
+     * @param reader GridMutablePortableReader
+     */
+    PortableLazyMapEntry(PortableBuilderReader reader) {
+        key = reader.parseValue();
+        val = reader.parseValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getKey() {
+        return PortableUtils.unwrapLazy(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getValue() {
+        return PortableUtils.unwrapLazy(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object setValue(Object val) {
+        Object res = getValue();
+
+        this.val = val;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        writer.writeByte(GridPortableMarshaller.MAP_ENTRY);
+
+        ctx.writeValue(writer, key);
+        ctx.writeValue(writer, val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
new file mode 100644
index 0000000..16772af
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazySet.java
@@ -0,0 +1,92 @@
+/*
+ * 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.builder;
+
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+class PortableLazySet extends PortableAbstractLazyValue {
+    /** */
+    private final int off;
+
+    /**
+     * @param reader Reader.
+     * @param size Size.
+     */
+    PortableLazySet(PortableBuilderReader reader, int size) {
+        super(reader, reader.position() - 1);
+
+        off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */;
+
+        assert size >= 0;
+
+        for (int i = 0; i < size; i++)
+            reader.skipValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val == 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 {
+            Collection<Object> c = (Collection<Object>)val;
+
+            writer.writeByte(GridPortableMarshaller.COL);
+            writer.writeInt(c.size());
+
+            byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */];
+            writer.writeByte(colType);
+
+            for (Object o : c)
+                ctx.writeValue(writer, o);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        int size = reader.readIntAbsolute(off + 1);
+
+        reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */);
+
+        Set<Object> res = U.newLinkedHashSet(size);
+
+        for (int i = 0; i < size; i++)
+            res.add(PortableUtils.unwrapLazy(reader.parseValue()));
+
+        return res;
+    }
+}
\ 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/builder/PortableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyValue.java
new file mode 100644
index 0000000..5d8d586
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableLazyValue.java
@@ -0,0 +1,28 @@
+/*
+ * 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.builder;
+
+/**
+ *
+ */
+public interface PortableLazyValue extends PortableBuilderSerializationAware {
+    /**
+     * @return Value.
+     */
+    public Object value();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
new file mode 100644
index 0000000..09fb844
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableModifiableLazyValue.java
@@ -0,0 +1,52 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+/**
+ *
+ */
+public class PortableModifiableLazyValue extends PortableAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader
+     * @param valOff
+     * @param len
+     */
+    public PortableModifiableLazyValue(PortableBuilderReader reader, int valOff, int len) {
+        super(reader, valOff);
+
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        return reader.reader().unmarshal(valOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val == null)
+            writer.write(reader.array(), valOff, len);
+        else
+            writer.writeObject(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
new file mode 100644
index 0000000..1126a3c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
@@ -0,0 +1,91 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.portable.PortableInvalidClassException;
+
+/**
+ *
+ */
+class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
+    /** */
+    private Object[] lazyValsArr;
+
+    /** */
+    private int compTypeId;
+
+    /** */
+    private String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected PortableObjectArrayLazyValue(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();
+
+        lazyValsArr = new Object[size];
+
+        for (int i = 0; i < size; i++)
+            lazyValsArr[i] = reader.parseValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        for (int i = 0; i < lazyValsArr.length; i++) {
+            if (lazyValsArr[i] instanceof PortableLazyValue)
+                lazyValsArr[i] = ((PortableLazyValue)lazyValsArr[i]).value();
+        }
+
+        return lazyValsArr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (clsName == null)
+            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
+        else
+            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName);
+    }
+}
\ 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/builder/PortablePlainLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
new file mode 100644
index 0000000..136958a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainLazyValue.java
@@ -0,0 +1,49 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.*;
+
+/**
+ *
+ */
+class PortablePlainLazyValue extends PortableAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader Reader
+     * @param valOff Offset
+     * @param len Length.
+     */
+    protected PortablePlainLazyValue(PortableBuilderReader reader, int valOff, int len) {
+        super(reader, valOff);
+
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object init() {
+        return reader.reader().unmarshal(valOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        writer.write(reader.array(), valOff, len);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
new file mode 100644
index 0000000..8743fbe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.builder;
+
+import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.portable.PortableObject;
+
+/**
+ *
+ */
+public class PortablePlainPortableObject implements PortableLazyValue {
+    /** */
+    private final PortableObject portableObj;
+
+    /**
+     * @param portableObj Portable object.
+     */
+    public PortablePlainPortableObject(PortableObject portableObj) {
+        this.portableObj = portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        return portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        PortableObject val = portableObj;
+
+        if (val instanceof PortableObjectOffheapImpl)
+            val = ((PortableObjectOffheapImpl)val).heapCopy();
+
+        writer.doWritePortableObject((PortableObjectImpl)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/builder/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
new file mode 100644
index 0000000..2e031f0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
@@ -0,0 +1,75 @@
+/*
+ * 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.builder;
+
+import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class PortableValueWithType implements PortableLazyValue {
+    /** */
+    private byte type;
+
+    /** */
+    private Object val;
+
+    /**
+     * @param type Type
+     * @param val Value.
+     */
+    PortableValueWithType(byte type, Object val) {
+        this.type = type;
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        if (val instanceof PortableBuilderSerializationAware)
+            ((PortableBuilderSerializationAware)val).writeTo(writer, ctx);
+        else
+            ctx.writeValue(writer, val);
+    }
+
+    /** {@inheritDoc} */
+    public String typeName() {
+        return CacheObjectPortableProcessorImpl.fieldTypeName(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        if (val instanceof PortableLazyValue)
+            return ((PortableLazyValue)val).value();
+
+        return val;
+    }
+
+    /**
+     * @param val New value.
+     */
+    public void value(Object val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableValueWithType.class, this);
+    }
+}
\ 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/builder/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
new file mode 100644
index 0000000..e069f3e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains classes specific to portable builder API.
+ */
+package org.apache.ignite.internal.portable.builder;
\ 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/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
index b33c643..1be5aea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
@@ -45,13 +45,13 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableMetaDataHandler;
 import org.apache.ignite.internal.portable.PortableMetaDataImpl;
 import org.apache.ignite.internal.portable.PortableObjectImpl;
 import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
+import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index d80cc49..dd8c3f3 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -281,7 +281,7 @@ org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager$1
 org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter
 org.apache.ignite.internal.portable.PortableClassDescriptor$Mode
 org.apache.ignite.internal.portable.PortableContext
-org.apache.ignite.internal.portable.PortableLazyMap$1$1$1
+org.apache.ignite.internal.portable.builder.PortableLazyMap$1$1$1
 org.apache.ignite.internal.portable.PortableMetaDataImpl
 org.apache.ignite.internal.portable.PortableObjectEx
 org.apache.ignite.internal.portable.PortableObjectImpl