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

[11/50] [abbrv] ignite git commit: ignite-2065: rename "portable" classes to "binary"

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
new file mode 100644
index 0000000..9b9e8b8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java
@@ -0,0 +1,846 @@
+/*
+ * 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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryPositionReadable;
+import org.apache.ignite.internal.binary.BinaryPrimitives;
+import org.apache.ignite.internal.binary.BinarySchema;
+import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
+import org.apache.ignite.internal.binary.BinaryUtils;
+
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ *
+ */
+public class BinaryBuilderReader implements BinaryPositionReadable {
+    /** */
+    private final BinaryContext ctx;
+
+    /** */
+    private final byte[] arr;
+
+    /** */
+    private final BinaryReaderExImpl reader;
+
+    /** */
+    private final Map<Integer, BinaryObjectBuilderImpl> objMap;
+
+    /** */
+    private int pos;
+
+    /*
+     * Constructor.
+     *
+     * @param objImpl Portable object
+     */
+    BinaryBuilderReader(BinaryObjectImpl objImpl) {
+        ctx = objImpl.context();
+        arr = objImpl.array();
+        pos = objImpl.start();
+
+        reader = new BinaryReaderExImpl(ctx,
+            BinaryHeapInputStream.create(arr, pos),
+            ctx.configuration().getClassLoader());
+
+        objMap = new HashMap<>();
+    }
+
+    /**
+     * Copying constructor.
+     *
+     * @param other Other reader.
+     * @param start Start position.
+     */
+    BinaryBuilderReader(BinaryBuilderReader other, int start) {
+        this.ctx = other.ctx;
+        this.arr = other.arr;
+        this.pos = start;
+
+        reader = new BinaryReaderExImpl(ctx, BinaryHeapInputStream.create(arr, start), null, other.reader.handles());
+
+        this.objMap = other.objMap;
+    }
+
+    /**
+     * @return Portable context.
+     */
+    public BinaryContext 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 BinarySchema 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 BinaryPrimitives.readInt(arr, pos + off);
+    }
+
+    /**
+     * @param pos Position in the source array.
+     * @return Read byte value.
+     */
+    public byte readBytePositioned(int pos) {
+        return BinaryPrimitives.readByte(arr, pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShortPositioned(int pos) {
+        return BinaryPrimitives.readShort(arr, pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readIntPositioned(int pos) {
+        return BinaryPrimitives.readInt(arr, pos);
+    }
+
+    /**
+     * @return Read length of array.
+     */
+    public int readLength() {
+        return BinaryPrimitives.readInt(arr, pos);
+    }
+
+    /**
+     * Read string length.
+     *
+     * @return String length.
+     */
+    public int readStringLength() {
+        return BinaryPrimitives.readInt(arr, pos);
+    }
+
+    /**
+     * Reads string.
+     *
+     * @return String.
+     */
+    public String readString() {
+        byte flag = readByte();
+
+        if (flag == GridBinaryMarshaller.NULL)
+            return null;
+
+        if (flag != GridBinaryMarshaller.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 GridBinaryMarshaller.NULL:
+                return;
+
+            case GridBinaryMarshaller.OBJ:
+                pos += readInt(GridBinaryMarshaller.TOTAL_LEN_POS - 1) - 1;
+
+                return;
+
+            case GridBinaryMarshaller.BOOLEAN:
+            case GridBinaryMarshaller.BYTE:
+                len = 1;
+                break;
+
+            case GridBinaryMarshaller.CHAR:
+            case GridBinaryMarshaller.SHORT:
+                len = 2;
+
+                break;
+
+            case GridBinaryMarshaller.HANDLE:
+            case GridBinaryMarshaller.FLOAT:
+            case GridBinaryMarshaller.INT:
+                len = 4;
+
+                break;
+
+            case GridBinaryMarshaller.ENUM:
+                //skipping type id and ordinal value
+                len = 8;
+
+                break;
+
+            case GridBinaryMarshaller.LONG:
+            case GridBinaryMarshaller.DOUBLE:
+                len = 8;
+
+                break;
+
+            case GridBinaryMarshaller.BYTE_ARR:
+            case GridBinaryMarshaller.BOOLEAN_ARR:
+                len = 4 + readLength();
+
+                break;
+
+            case GridBinaryMarshaller.STRING:
+                len = 4 + readStringLength();
+
+                break;
+
+            case GridBinaryMarshaller.DECIMAL:
+                len = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
+
+                break;
+
+            case GridBinaryMarshaller.UUID:
+                len = 8 + 8;
+
+                break;
+
+            case GridBinaryMarshaller.DATE:
+                len = 8;
+
+                break;
+
+            case GridBinaryMarshaller.TIMESTAMP:
+                len = 8 + 4;
+
+                break;
+
+            case GridBinaryMarshaller.CHAR_ARR:
+            case GridBinaryMarshaller.SHORT_ARR:
+                len = 4 + readLength() * 2;
+
+                break;
+
+            case GridBinaryMarshaller.INT_ARR:
+            case GridBinaryMarshaller.FLOAT_ARR:
+                len = 4 + readLength() * 4;
+
+                break;
+
+            case GridBinaryMarshaller.LONG_ARR:
+            case GridBinaryMarshaller.DOUBLE_ARR:
+                len = 4 + readLength() * 8;
+
+                break;
+
+            case GridBinaryMarshaller.DECIMAL_ARR:
+            case GridBinaryMarshaller.DATE_ARR:
+            case GridBinaryMarshaller.TIMESTAMP_ARR:
+            case GridBinaryMarshaller.OBJ_ARR:
+            case GridBinaryMarshaller.ENUM_ARR:
+            case GridBinaryMarshaller.UUID_ARR:
+            case GridBinaryMarshaller.STRING_ARR: {
+                int size = readInt();
+
+                for (int i = 0; i < size; i++)
+                    skipValue();
+
+                return;
+            }
+
+            case GridBinaryMarshaller.COL: {
+                int size = readInt();
+
+                pos++; // skip collection type
+
+                for (int i = 0; i < size; i++)
+                    skipValue();
+
+                return;
+            }
+
+            case GridBinaryMarshaller.MAP: {
+                int size = readInt();
+
+                pos++; // skip collection type
+
+                for (int i = 0; i < size; i++) {
+                    skipValue(); // skip key.
+                    skipValue(); // skip value.
+                }
+
+                return;
+            }
+
+            case GridBinaryMarshaller.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 GridBinaryMarshaller.NULL:
+                return null;
+
+            case GridBinaryMarshaller.HANDLE: {
+                int objStart = pos - readIntPositioned(pos + 1);
+
+                BinaryObjectBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new BinaryBuilderReader(this, objStart), objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridBinaryMarshaller.OBJ: {
+                BinaryObjectBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new BinaryBuilderReader(this, pos), pos);
+
+                    objMap.put(pos, res);
+                }
+
+                return res;
+            }
+
+            case GridBinaryMarshaller.BYTE:
+                return arr[pos + 1];
+
+            case GridBinaryMarshaller.SHORT:
+                return BinaryPrimitives.readShort(arr, pos + 1);
+
+            case GridBinaryMarshaller.INT:
+                return BinaryPrimitives.readInt(arr, pos + 1);
+
+            case GridBinaryMarshaller.LONG:
+                return BinaryPrimitives.readLong(arr, pos + 1);
+
+            case GridBinaryMarshaller.FLOAT:
+                return BinaryPrimitives.readFloat(arr, pos + 1);
+
+            case GridBinaryMarshaller.DOUBLE:
+                return BinaryPrimitives.readDouble(arr, pos + 1);
+
+            case GridBinaryMarshaller.CHAR:
+                return BinaryPrimitives.readChar(arr, pos + 1);
+
+            case GridBinaryMarshaller.BOOLEAN:
+                return arr[pos + 1] != 0;
+
+            case GridBinaryMarshaller.DECIMAL:
+            case GridBinaryMarshaller.STRING:
+            case GridBinaryMarshaller.UUID:
+            case GridBinaryMarshaller.DATE:
+            case GridBinaryMarshaller.TIMESTAMP:
+                return new BinaryPlainLazyValue(this, pos, len);
+
+            case GridBinaryMarshaller.BYTE_ARR:
+            case GridBinaryMarshaller.SHORT_ARR:
+            case GridBinaryMarshaller.INT_ARR:
+            case GridBinaryMarshaller.LONG_ARR:
+            case GridBinaryMarshaller.FLOAT_ARR:
+            case GridBinaryMarshaller.DOUBLE_ARR:
+            case GridBinaryMarshaller.CHAR_ARR:
+            case GridBinaryMarshaller.BOOLEAN_ARR:
+            case GridBinaryMarshaller.DECIMAL_ARR:
+            case GridBinaryMarshaller.DATE_ARR:
+            case GridBinaryMarshaller.TIMESTAMP_ARR:
+            case GridBinaryMarshaller.UUID_ARR:
+            case GridBinaryMarshaller.STRING_ARR:
+            case GridBinaryMarshaller.ENUM_ARR:
+            case GridBinaryMarshaller.OBJ_ARR:
+            case GridBinaryMarshaller.COL:
+            case GridBinaryMarshaller.MAP:
+                return new LazyCollection(pos);
+
+            case GridBinaryMarshaller.ENUM: {
+                if (len == 1) {
+                    assert readByte(pos) == GridBinaryMarshaller.NULL;
+
+                    return null;
+                }
+
+                int mark = position();
+                position(pos + 1);
+
+                BinaryBuilderEnum builderEnum = new BinaryBuilderEnum(this);
+
+                position(mark);
+
+                return builderEnum;
+            }
+
+            case GridBinaryMarshaller.PORTABLE_OBJ: {
+                int size = readIntPositioned(pos + 1);
+
+                int start = readIntPositioned(pos + 4 + size);
+
+                BinaryObjectImpl portableObj = new BinaryObjectImpl(ctx, arr, pos + 4 + start);
+
+                return new BinaryPlainBinaryObject(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 GridBinaryMarshaller.NULL:
+                return null;
+
+            case GridBinaryMarshaller.HANDLE: {
+                int objStart = pos - 1 - readInt();
+
+                BinaryObjectBuilderImpl res = objMap.get(objStart);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new BinaryBuilderReader(this, objStart), objStart);
+
+                    objMap.put(objStart, res);
+                }
+
+                return res;
+            }
+
+            case GridBinaryMarshaller.OBJ: {
+                pos--;
+
+                BinaryObjectBuilderImpl res = objMap.get(pos);
+
+                if (res == null) {
+                    res = new BinaryObjectBuilderImpl(new BinaryBuilderReader(this, pos), pos);
+
+                    objMap.put(pos, res);
+                }
+
+                pos += readInt(GridBinaryMarshaller.TOTAL_LEN_POS);
+
+                return res;
+            }
+
+            case GridBinaryMarshaller.BYTE:
+                return arr[pos++];
+
+            case GridBinaryMarshaller.SHORT: {
+                Object res = BinaryPrimitives.readShort(arr, pos);
+                pos += 2;
+                return res;
+            }
+
+            case GridBinaryMarshaller.INT:
+                return readInt();
+
+            case GridBinaryMarshaller.LONG:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridBinaryMarshaller.FLOAT:
+                plainLazyValLen = 4;
+
+                break;
+
+            case GridBinaryMarshaller.DOUBLE:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridBinaryMarshaller.CHAR:
+                plainLazyValLen = 2;
+
+                break;
+
+            case GridBinaryMarshaller.BOOLEAN:
+                return arr[pos++] != 0;
+
+            case GridBinaryMarshaller.DECIMAL:
+                plainLazyValLen = /** scale */ 4  + /** mag len */ 4  + /** mag bytes count */ readInt(4);
+
+                break;
+
+            case GridBinaryMarshaller.STRING:
+                plainLazyValLen = 4 + readStringLength();
+
+                break;
+
+            case GridBinaryMarshaller.UUID:
+                plainLazyValLen = 8 + 8;
+
+                break;
+
+            case GridBinaryMarshaller.DATE:
+                plainLazyValLen = 8;
+
+                break;
+
+            case GridBinaryMarshaller.TIMESTAMP:
+                plainLazyValLen = 8 + 4;
+
+                break;
+
+            case GridBinaryMarshaller.BYTE_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.SHORT_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.INT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.LONG_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.FLOAT_ARR:
+                plainLazyValLen = 4 + readLength() * 4;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.DOUBLE_ARR:
+                plainLazyValLen = 4 + readLength() * 8;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.CHAR_ARR:
+                plainLazyValLen = 4 + readLength() * 2;
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.BOOLEAN_ARR:
+                plainLazyValLen = 4 + readLength();
+                modifiableLazyVal = true;
+
+                break;
+
+            case GridBinaryMarshaller.OBJ_ARR:
+                return new BinaryObjectArrayLazyValue(this);
+
+            case GridBinaryMarshaller.DATE_ARR: {
+                int size = readInt();
+
+                Date[] res = new Date[size];
+
+                for (int i = 0; i < res.length; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridBinaryMarshaller.NULL) continue;
+
+                    if (flag != GridBinaryMarshaller.DATE)
+                        throw new BinaryObjectException("Invalid flag value: " + flag);
+
+                    long time = BinaryPrimitives.readLong(arr, pos);
+
+                    pos += 8;
+
+                    res[i] = new Date(time);
+                }
+
+                return res;
+            }
+
+            case GridBinaryMarshaller.TIMESTAMP_ARR: {
+                int size = readInt();
+
+                Timestamp[] res = new Timestamp[size];
+
+                for (int i = 0; i < res.length; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridBinaryMarshaller.NULL)
+                        continue;
+
+                    if (flag != GridBinaryMarshaller.TIMESTAMP)
+                        throw new BinaryObjectException("Invalid flag value: " + flag);
+
+                    long time = BinaryPrimitives.readLong(arr, pos);
+
+                    pos += 8;
+
+                    int nano = BinaryPrimitives.readInt(arr, pos);
+
+                    pos += 4;
+
+                    Timestamp ts = new Timestamp(time);
+
+                    ts.setNanos(ts.getNanos() + nano);
+
+                    res[i] = ts;
+                }
+
+                return res;
+            }
+
+            case GridBinaryMarshaller.UUID_ARR:
+            case GridBinaryMarshaller.STRING_ARR:
+            case GridBinaryMarshaller.DECIMAL_ARR: {
+                int size = readInt();
+
+                for (int i = 0; i < size; i++) {
+                    byte flag = arr[pos++];
+
+                    if (flag == GridBinaryMarshaller.UUID)
+                        pos += 8 + 8;
+                    else if (flag == GridBinaryMarshaller.STRING)
+                        pos += 4 + readStringLength();
+                    else if (flag == GridBinaryMarshaller.DECIMAL) {
+                        pos += 4; // scale value
+                        pos += 4 + readLength();
+                    }
+                    else
+                        assert flag == GridBinaryMarshaller.NULL;
+                }
+
+                return new BinaryModifiableLazyValue(this, valPos, pos - valPos);
+            }
+
+            case GridBinaryMarshaller.COL: {
+                int size = readInt();
+                byte colType = arr[pos++];
+
+                switch (colType) {
+                    case GridBinaryMarshaller.USER_COL:
+                    case GridBinaryMarshaller.ARR_LIST:
+                        return new BinaryLazyArrayList(this, size);
+
+                    case GridBinaryMarshaller.LINKED_LIST:
+                        return new BinaryLazyLinkedList(this, size);
+
+                    case GridBinaryMarshaller.HASH_SET:
+                    case GridBinaryMarshaller.LINKED_HASH_SET:
+                        return new BinaryLazySet(this, size);
+                }
+
+                throw new BinaryObjectException("Unknown collection type: " + colType);
+            }
+
+            case GridBinaryMarshaller.MAP:
+                return BinaryLazyMap.parseMap(this);
+
+            case GridBinaryMarshaller.ENUM:
+                return new BinaryBuilderEnum(this);
+
+            case GridBinaryMarshaller.ENUM_ARR:
+                return new BinaryEnumArrayLazyValue(this);
+
+            case GridBinaryMarshaller.PORTABLE_OBJ: {
+                int size = readInt();
+
+                pos += size;
+
+                int start = readInt();
+
+                BinaryObjectImpl portableObj = new BinaryObjectImpl(ctx, arr,
+                    pos - 4 - size + start);
+
+                return new BinaryPlainBinaryObject(portableObj);
+            }
+
+            default:
+                throw new BinaryObjectException("Invalid flag value: " + type);
+        }
+
+        BinaryAbstractLazyValue res;
+
+        if (modifiableLazyVal)
+            res = new BinaryModifiableLazyValue(this, valPos, 1 + plainLazyValLen);
+        else
+            res = new BinaryPlainLazyValue(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 BinaryLazyValue {
+        /** */
+        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, BinaryBuilderSerializer ctx) {
+            ctx.writeValue(writer, wrappedCollection());
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object value() {
+            return BinaryUtils.unwrapLazy(wrappedCollection());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializationAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializationAware.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializationAware.java
new file mode 100644
index 0000000..7710ba3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializationAware.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 BinaryBuilderSerializationAware {
+    /**
+     * @param writer Writer.
+     * @param ctx Context.
+     */
+    public void writeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer ctx);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
new file mode 100644
index 0000000..458602d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryUtils;
+
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ *
+ */
+class BinaryBuilderSerializer {
+    /** */
+    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(GridBinaryMarshaller.NULL);
+
+            return;
+        }
+
+        if (val instanceof BinaryBuilderSerializationAware) {
+            ((BinaryBuilderSerializationAware)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(GridBinaryMarshaller.HANDLE);
+                writer.writeInt(handle);
+            }
+
+            return;
+        }
+
+        if (val.getClass().isEnum()) {
+            String typeName = BinaryContext.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(GridBinaryMarshaller.ENUM);
+            writer.writeInt(typeId);
+            writer.writeInt(((Enum)val).ordinal());
+
+            return;
+        }
+
+        if (val instanceof Collection) {
+            Collection<?> c = (Collection<?>)val;
+
+            writer.writeByte(GridBinaryMarshaller.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(GridBinaryMarshaller.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 = BinaryUtils.PLAIN_CLASS_TO_FLAG.get(val.getClass());
+
+        if (flag != null) {
+            BinaryUtils.writePlainObject(writer, val);
+
+            return;
+        }
+
+        if (val instanceof Object[]) {
+            int compTypeId = writer.context().typeId(((Object[])val).getClass().getComponentType().getName());
+
+            if (val instanceof BinaryBuilderEnum[]) {
+                writeArray(writer, GridBinaryMarshaller.ENUM_ARR, (Object[])val, compTypeId);
+
+                return;
+            }
+
+            if (((Object[])val).getClass().getComponentType().isEnum()) {
+                Enum[] enumArr = (Enum[])val;
+
+                writer.writeByte(GridBinaryMarshaller.ENUM_ARR);
+                writer.writeInt(compTypeId);
+                writer.writeInt(enumArr.length);
+
+                for (Enum anEnum : enumArr)
+                    writeValue(writer, anEnum);
+
+                return;
+            }
+
+            writeArray(writer, GridBinaryMarshaller.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(GridBinaryMarshaller.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/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java
new file mode 100644
index 0000000..db55050
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java
@@ -0,0 +1,113 @@
+/*
+ * 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.GridBinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+
+/**
+ *
+ */
+class BinaryEnumArrayLazyValue extends BinaryAbstractLazyValue {
+    /** */
+    private final int len;
+
+    /** */
+    private final int compTypeId;
+
+    /** */
+    private final String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected BinaryEnumArrayLazyValue(BinaryBuilderReader reader) {
+        super(reader, reader.position() - 1);
+
+        int typeId = reader.readInt();
+
+        if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                cls = U.forName(reader.readString(), reader.portableContext().configuration().getClassLoader());
+            }
+            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();
+
+        BinaryBuilderEnum[] res = new BinaryBuilderEnum[size];
+
+        for (int i = 0; i < size; i++) {
+            byte flag = reader.readByte();
+
+            if (flag == GridBinaryMarshaller.NULL)
+                continue;
+
+            if (flag != GridBinaryMarshaller.ENUM)
+                throw new BinaryObjectException("Invalid flag value: " + flag);
+
+            res[i] = new BinaryBuilderEnum(reader);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer ctx) {
+        if (val != null) {
+            if (clsName != null)
+                ctx.writeArray(writer, GridBinaryMarshaller.ENUM_ARR, (Object[])val, clsName);
+            else
+                ctx.writeArray(writer, GridBinaryMarshaller.ENUM_ARR, (Object[])val, compTypeId);
+
+            return;
+        }
+
+        writer.write(reader.array(), valOff, len);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyArrayList.java
new file mode 100644
index 0000000..3627b1d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyArrayList.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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryUtils;
+
+/**
+ *
+ */
+class BinaryLazyArrayList extends AbstractList<Object> implements BinaryBuilderSerializationAware {
+    /** */
+    private final BinaryBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private List<Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param size Size,
+     */
+    BinaryLazyArrayList(BinaryBuilderReader 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 BinaryUtils.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 BinaryUtils.unwrapLazy(delegate.set(idx, element));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(int idx) {
+        ensureDelegateInit();
+
+        return BinaryUtils.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, BinaryBuilderSerializer 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(GridBinaryMarshaller.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/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyLinkedList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyLinkedList.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyLinkedList.java
new file mode 100644
index 0000000..1813fcc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyLinkedList.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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryUtils;
+
+/**
+ *
+ */
+class BinaryLazyLinkedList extends AbstractList<Object> implements BinaryBuilderSerializationAware {
+    /** */
+    private final BinaryBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private List<Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param size Size,
+     */
+    BinaryLazyLinkedList(BinaryBuilderReader 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 BinaryUtils.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 BinaryUtils.unwrapLazy(delegate.set(idx, element));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(int idx) {
+        ensureDelegateInit();
+
+        return BinaryUtils.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 = BinaryLazyLinkedList.super.listIterator(idx);
+
+            @Override public boolean hasNext() {
+                return delegate.hasNext();
+            }
+
+            @Override public Object next() {
+                return BinaryUtils.unwrapLazy(delegate.next());
+            }
+
+            @Override public boolean hasPrevious() {
+                return delegate.hasPrevious();
+            }
+
+            @Override public Object previous() {
+                return BinaryUtils.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 BinaryUtils.unwrapLazyIterator(super.iterator());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer 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(GridBinaryMarshaller.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/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyMap.java
new file mode 100644
index 0000000..c82ff7d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyMap.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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class BinaryLazyMap extends AbstractMap<Object, Object> implements BinaryBuilderSerializationAware {
+    /** */
+    private final BinaryBuilderReader reader;
+
+    /** */
+    private final int off;
+
+    /** */
+    private Map<Object, Object> delegate;
+
+    /**
+     * @param reader Reader.
+     * @param off Offset.
+     */
+    private BinaryLazyMap(BinaryBuilderReader reader, int off) {
+        this.reader = reader;
+        this.off = off;
+    }
+
+    /**
+     * @param reader Reader.
+     * @return PortableLazyMap.
+     */
+    @Nullable public static BinaryLazyMap parseMap(BinaryBuilderReader 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 BinaryLazyMap(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(BinaryUtils.unwrapLazy(reader.parseValue()), reader.parseValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer 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(GridBinaryMarshaller.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 BinaryUtils.unwrapLazy(delegate.get(key));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object put(Object key, Object val) {
+        ensureDelegateInit();
+
+        return BinaryUtils.unwrapLazy(delegate.put(key, val));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object remove(Object key) {
+        ensureDelegateInit();
+
+        return BinaryUtils.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 BinaryLazyValue) {
+                            return new SimpleEntry<Object, Object>(res.getKey(), val) {
+                                private static final long serialVersionUID = 0L;
+
+                                @Override public Object getValue() {
+                                    return ((BinaryLazyValue)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/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazySet.java
new file mode 100644
index 0000000..6be4c76
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazySet.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.binary.builder;
+
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+class BinaryLazySet extends BinaryAbstractLazyValue {
+    /** */
+    private final int off;
+
+    /**
+     * @param reader Reader.
+     * @param size Size.
+     */
+    BinaryLazySet(BinaryBuilderReader 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(BinaryWriterExImpl writer, BinaryBuilderSerializer ctx) {
+        if (val == 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 {
+            Collection<Object> c = (Collection<Object>)val;
+
+            writer.writeByte(GridBinaryMarshaller.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.readIntPositioned(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(BinaryUtils.unwrapLazy(reader.parseValue()));
+
+        return res;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyValue.java
new file mode 100644
index 0000000..f9da84b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryLazyValue.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.binary.builder;
+
+/**
+ *
+ */
+public interface BinaryLazyValue extends BinaryBuilderSerializationAware {
+    /**
+     * @return Value.
+     */
+    public Object value();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryModifiableLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryModifiableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryModifiableLazyValue.java
new file mode 100644
index 0000000..070abf3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryModifiableLazyValue.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.binary.builder;
+
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+
+/**
+ *
+ */
+public class BinaryModifiableLazyValue extends BinaryAbstractLazyValue {
+    /** */
+    protected final int len;
+
+    /**
+     * @param reader
+     * @param valOff
+     * @param len
+     */
+    public BinaryModifiableLazyValue(BinaryBuilderReader 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(BinaryWriterExImpl writer, BinaryBuilderSerializer ctx) {
+        if (val == null)
+            writer.write(reader.array(), valOff, len);
+        else
+            writer.writeObject(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java
new file mode 100644
index 0000000..05713b4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java
@@ -0,0 +1,90 @@
+/*
+ * 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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+
+/**
+ *
+ */
+class BinaryObjectArrayLazyValue extends BinaryAbstractLazyValue {
+    /** */
+    private Object[] lazyValsArr;
+
+    /** */
+    private int compTypeId;
+
+    /** */
+    private String clsName;
+
+    /**
+     * @param reader Reader.
+     */
+    protected BinaryObjectArrayLazyValue(BinaryBuilderReader reader) {
+        super(reader, reader.position() - 1);
+
+        int typeId = reader.readInt();
+
+        if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
+            clsName = reader.readString();
+
+            Class cls;
+
+            try {
+                cls = U.forName(reader.readString(), reader.portableContext().configuration().getClassLoader());
+            }
+            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();
+
+        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 BinaryLazyValue)
+                lazyValsArr[i] = ((BinaryLazyValue)lazyValsArr[i]).value();
+        }
+
+        return lazyValsArr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer ctx) {
+        if (clsName == null)
+            ctx.writeArray(writer, GridBinaryMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
+        else
+            ctx.writeArray(writer, GridBinaryMarshaller.OBJ_ARR, lazyValsArr, clsName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/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
index f1d4185..5e60a20 100644
--- 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
@@ -25,19 +25,12 @@ 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.GridBinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinarySchema;
+import org.apache.ignite.internal.binary.BinarySchemaRegistry;
 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.binary.BinaryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -49,13 +42,6 @@ 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;
-
 /**
  *
  */
@@ -64,7 +50,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     private static final Object REMOVED_FIELD_MARKER = new Object();
 
     /** */
-    private final PortableContext ctx;
+    private final BinaryContext ctx;
 
     /** */
     private final int typeId;
@@ -94,7 +80,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     private final int hdrLen;
 
     /** Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject. */
-    private final PortableBuilderReader reader;
+    private final BinaryBuilderReader reader;
 
     /** */
     private int hashCode;
@@ -103,8 +89,8 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param clsName Class name.
      * @param ctx Portable context.
      */
-    public BinaryObjectBuilderImpl(PortableContext ctx, String clsName) {
-        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
+    public BinaryObjectBuilderImpl(BinaryContext ctx, String clsName) {
+        this(ctx, ctx.typeId(clsName), BinaryContext.typeName(clsName));
     }
 
     /**
@@ -112,7 +98,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param ctx Context.
      * @param typeId Type id.
      */
-    public BinaryObjectBuilderImpl(PortableContext ctx, int typeId, String typeName) {
+    public BinaryObjectBuilderImpl(BinaryContext ctx, int typeId, String typeName) {
         this.typeId = typeId;
         this.typeName = typeName;
         this.ctx = ctx;
@@ -120,7 +106,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         start = -1;
         flags = -1;
         reader = null;
-        hdrLen = GridPortableMarshaller.DFLT_HDR_LEN;
+        hdrLen = GridBinaryMarshaller.DFLT_HDR_LEN;
 
         readCache = Collections.emptyMap();
     }
@@ -129,7 +115,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param obj Object to wrap.
      */
     public BinaryObjectBuilderImpl(BinaryObjectImpl obj) {
-        this(new PortableBuilderReader(obj), obj.start());
+        this(new BinaryBuilderReader(obj), obj.start());
 
         reader.registerObject(this);
     }
@@ -138,23 +124,23 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param reader ctx
      * @param start Start.
      */
-    BinaryObjectBuilderImpl(PortableBuilderReader reader, int start) {
+    BinaryObjectBuilderImpl(BinaryBuilderReader reader, int start) {
         this.reader = reader;
         this.start = start;
-        this.flags = reader.readShortPositioned(start + GridPortableMarshaller.FLAGS_POS);
+        this.flags = reader.readShortPositioned(start + GridBinaryMarshaller.FLAGS_POS);
 
-        byte ver = reader.readBytePositioned(start + GridPortableMarshaller.PROTO_VER_POS);
+        byte ver = reader.readBytePositioned(start + GridBinaryMarshaller.PROTO_VER_POS);
 
-        PortableUtils.checkProtocolVersion(ver);
+        BinaryUtils.checkProtocolVersion(ver);
 
-        int typeId = reader.readIntPositioned(start + GridPortableMarshaller.TYPE_ID_POS);
+        int typeId = reader.readIntPositioned(start + GridBinaryMarshaller.TYPE_ID_POS);
         ctx = reader.portableContext();
-        hashCode = reader.readIntPositioned(start + GridPortableMarshaller.HASH_CODE_POS);
+        hashCode = reader.readIntPositioned(start + GridBinaryMarshaller.HASH_CODE_POS);
 
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
+        if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
             int mark = reader.position();
 
-            reader.position(start + GridPortableMarshaller.DFLT_HDR_LEN);
+            reader.position(start + GridBinaryMarshaller.DFLT_HDR_LEN);
 
             clsNameToWrite = reader.readString();
 
@@ -177,7 +163,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         }
         else {
             this.typeId = typeId;
-            hdrLen = GridPortableMarshaller.DFLT_HDR_LEN;
+            hdrLen = GridBinaryMarshaller.DFLT_HDR_LEN;
         }
     }
 
@@ -186,7 +172,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx)) {
             writer.typeId(typeId);
 
-            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
+            BinaryBuilderSerializer serializationCtx = new BinaryBuilderSerializer();
 
             serializationCtx.registerObjectWriting(this, 0);
 
@@ -202,14 +188,14 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param writer Writer.
      * @param serializer Serializer.
      */
-    void serializeTo(BinaryWriterExImpl writer, PortableBuilderSerializer serializer) {
+    void serializeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer serializer) {
         try {
             writer.preWrite(registeredType ? null : clsNameToWrite);
 
             Set<Integer> remainsFlds = null;
 
             if (reader != null) {
-                PortableSchema schema = reader.schema();
+                BinarySchema schema = reader.schema();
 
                 Map<Integer, Object> assignedFldsById;
 
@@ -228,16 +214,16 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                     assignedFldsById = Collections.emptyMap();
 
                 // Get footer details.
-                int fieldIdLen = PortableUtils.fieldIdLength(flags);
-                int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+                int fieldIdLen = BinaryUtils.fieldIdLength(flags);
+                int fieldOffsetLen = BinaryUtils.fieldOffsetLength(flags);
 
-                IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start);
+                IgniteBiTuple<Integer, Integer> footer = BinaryUtils.footerAbsolute(reader, start);
 
                 int footerPos = footer.get1();
                 int footerEnd = footer.get2();
 
                 // Get raw position.
-                int rawPos = PortableUtils.rawOffsetAbsolute(reader, start);
+                int rawPos = BinaryUtils.rawOffsetAbsolute(reader, start);
 
                 // Position reader on data.
                 reader.position(start + hdrLen);
@@ -265,7 +251,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                     else {
                         int type = fieldLen != 0 ? reader.readByte(0) : 0;
 
-                        if (fieldLen != 0 && !PortableUtils.isPlainArrayType(type) && PortableUtils.isPlainType(type)) {
+                        if (fieldLen != 0 && !BinaryUtils.isPlainArrayType(type) && BinaryUtils.isPlainType(type)) {
                             writer.writeFieldId(fieldId);
 
                             writer.write(reader.array(), reader.position(), fieldLen);
@@ -321,26 +307,26 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
                     int newFldTypeId;
 
-                    if (val instanceof PortableValueWithType) {
-                        newFldTypeId = ((PortableValueWithType)val).typeId();
+                    if (val instanceof BinaryValueWithType) {
+                        newFldTypeId = ((BinaryValueWithType)val).typeId();
 
-                        if (newFldTypeId == GridPortableMarshaller.OBJ && ((PortableValueWithType)val).value() == null)
+                        if (newFldTypeId == GridBinaryMarshaller.OBJ && ((BinaryValueWithType)val).value() == null)
                             nullObjField = true;
                     }
                     else
-                        newFldTypeId = PortableUtils.typeByClass(val.getClass());
+                        newFldTypeId = BinaryUtils.typeByClass(val.getClass());
 
-                    String newFldTypeName = PortableUtils.fieldTypeName(newFldTypeId);
+                    String newFldTypeName = BinaryUtils.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));
+                        fieldsMeta.put(name, BinaryUtils.fieldTypeId(newFldTypeName));
                     }
                     else if (!nullObjField) {
-                        String objTypeName = PortableUtils.fieldTypeName(GridPortableMarshaller.OBJ);
+                        String objTypeName = BinaryUtils.fieldTypeName(GridBinaryMarshaller.OBJ);
 
                         if (!objTypeName.equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
                             throw new BinaryObjectException(
@@ -357,8 +343,8 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
             if (reader != null) {
                 // Write raw data if any.
-                int rawOff = PortableUtils.rawOffsetAbsolute(reader, start);
-                int footerStart = PortableUtils.footerStartAbsolute(reader, start);
+                int rawOff = BinaryUtils.rawOffsetAbsolute(reader, start);
+                int footerStart = BinaryUtils.footerStartAbsolute(reader, start);
 
                 if (rawOff < footerStart) {
                     writer.rawWriter();
@@ -367,7 +353,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                 }
 
                 // Shift reader to the end of the object.
-                reader.position(start + PortableUtils.length(reader, start));
+                reader.position(start + BinaryUtils.length(reader, start));
             }
 
             writer.postWrite(true, registeredType, hashCode);
@@ -375,7 +361,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
             // Update metadata if needed.
             int schemaId = writer.schemaId();
 
-            PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
+            BinarySchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
 
             if (schemaReg.schema(schemaId) == null) {
                 String typeName = this.typeName;
@@ -386,7 +372,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                     typeName = meta.typeName();
                 }
 
-                PortableSchema curSchema = writer.currentSchema();
+                BinarySchema curSchema = writer.currentSchema();
 
                 ctx.updateMetadata(typeId, new BinaryMetadata(typeId, typeName, fieldsMeta,
                     ctx.affinityKeyFieldName(typeId), Collections.singleton(curSchema), false));
@@ -419,7 +405,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     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 fieldOffset = BinaryUtils.fieldOffsetRelative(reader, footerPos + fieldIdLen, fieldOffsetLen);
         int fieldPos = start + fieldOffset;
 
         // Get field length.
@@ -430,7 +416,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
             fieldLen = rawPos - fieldPos;
         else {
             // Field is somewhere in the middle, get difference with the next offset.
-            int nextFieldOffset = PortableUtils.fieldOffsetRelative(reader,
+            int nextFieldOffset = BinaryUtils.fieldOffsetRelative(reader,
                 footerPos + fieldIdLen + fieldOffsetLen + fieldIdLen, fieldOffsetLen);
 
             fieldLen = nextFieldOffset - fieldOffset;
@@ -446,19 +432,19 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         assert reader != null;
 
         if (readCache == null) {
-            int fieldIdLen = PortableUtils.fieldIdLength(flags);
-            int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+            int fieldIdLen = BinaryUtils.fieldIdLength(flags);
+            int fieldOffsetLen = BinaryUtils.fieldOffsetLength(flags);
 
-            PortableSchema schema = reader.schema();
+            BinarySchema schema = reader.schema();
 
             Map<Integer, Object> readCache = new HashMap<>();
 
-            IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start);
+            IgniteBiTuple<Integer, Integer> footer = BinaryUtils.footerAbsolute(reader, start);
 
             int footerPos = footer.get1();
             int footerEnd = footer.get2();
 
-            int rawPos = PortableUtils.rawOffsetAbsolute(reader, start);
+            int rawPos = BinaryUtils.rawOffsetAbsolute(reader, start);
 
             int idx = 0;
 
@@ -499,20 +485,20 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
             val = readCache.get(fldId);
         }
 
-        return (T)PortableUtils.unwrapLazy(val);
+        return (T)BinaryUtils.unwrapLazy(val);
     }
 
     /** {@inheritDoc} */
     @Override public BinaryObjectBuilder setField(String name, Object val0) {
-        Object val = val0 == null ? new PortableValueWithType(PortableUtils.typeByClass(Object.class), null) : val0;
+        Object val = val0 == null ? new BinaryValueWithType(BinaryUtils.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);
+        if (oldVal instanceof BinaryValueWithType && val0 != null) {
+            ((BinaryValueWithType)oldVal).value(val);
 
             assignedVals.put(name, oldVal);
         }
@@ -525,7 +511,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         if (assignedVals == null)
             assignedVals = new LinkedHashMap<>();
 
-        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
+        assignedVals.put(name, new BinaryValueWithType(BinaryUtils.typeByClass(type), val));
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryPlainBinaryObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryPlainBinaryObject.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryPlainBinaryObject.java
new file mode 100644
index 0000000..5ac4e97
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryPlainBinaryObject.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.binary.builder;
+
+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.binary.BinaryObject;
+
+/**
+ *
+ */
+public class BinaryPlainBinaryObject implements BinaryLazyValue {
+    /** */
+    private final BinaryObject portableObj;
+
+    /**
+     * @param portableObj Portable object.
+     */
+    public BinaryPlainBinaryObject(BinaryObject portableObj) {
+        this.portableObj = portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        return portableObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer ctx) {
+        BinaryObject val = portableObj;
+
+        if (val instanceof BinaryObjectOffheapImpl)
+            val = ((BinaryObjectOffheapImpl)val).heapCopy();
+
+        writer.doWritePortableObject((BinaryObjectImpl)val);
+    }
+}